From ac88f255283f4367c8a86fe9f9ab7f7ab66bb697 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Sat, 27 Aug 2016 13:36:18 +0800 Subject: [PATCH 01/25] fix path --- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index e32d30178eeb..1eadde223b91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -390,9 +390,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { + val str = ctx.path.getText + val pathStr = str.substring(1, str.length - 1) LoadDataCommand( table = visitTableIdentifier(ctx.tableIdentifier), - path = string(ctx.path), + path = pathStr, isLocal = ctx.LOCAL != null, isOverwrite = ctx.OVERWRITE != null, partition = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) @@ -829,7 +831,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * Create location string. */ override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) { - string(ctx.STRING) +// string(ctx.STRING) + val str = ctx.STRING().getText + str.substring(1, str.length - 1) } /** From a3af25abbbe16ca71cabfb5f31238b19cab6a3b0 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Thu, 25 Aug 2016 17:28:48 +0800 Subject: [PATCH 02/25] refactor table statistics --- .../catalyst/catalog/ExternalCatalog.scala | 2 + .../catalyst/catalog/InMemoryCatalog.scala | 2 + .../sql/catalyst/catalog/SessionCatalog.scala | 8 +- .../sql/catalyst/catalog/interface.scala | 3 +- .../catalyst/plans/logical/Statistics.scala | 6 +- .../spark/sql/execution/SparkSqlParser.scala | 4 +- .../command/AnalyzeTableCommand.scala | 79 +++++++++++++++--- .../datasources/DataSourceStrategy.scala | 4 +- .../datasources/LogicalRelation.scala | 9 +- .../spark/sql/internal/SessionState.scala | 4 +- .../apache/spark/sql/StatisticsSuite.scala | 28 +++++++ .../spark/sql/hive/HiveExternalCatalog.scala | 7 ++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 +- .../spark/sql/hive/MetastoreRelation.scala | 11 ++- .../spark/sql/hive/client/HiveClient.scala | 4 + .../sql/hive/client/HiveClientImpl.scala | 40 +++++++++ .../spark/sql/hive/StatisticsSuite.scala | 83 +++++++++++++++++++ 17 files changed, 271 insertions(+), 31 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 27e1810814c6..8c53dd926ed2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -85,6 +85,8 @@ abstract class ExternalCatalog { */ def alterTable(tableDefinition: CatalogTable): Unit + def alterTableStats(tableDefinition: CatalogTable): Unit + def getTable(db: String, table: String): CatalogTable def getTableOption(db: String, table: String): Option[CatalogTable] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index b55ddcb54b45..0b6fc5963a34 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -278,6 +278,8 @@ class InMemoryCatalog( catalog(db).tables(tableDefinition.identifier.table).table = tableDefinition } + override def alterTableStats(tableDefinition: CatalogTable): Unit = alterTable(tableDefinition) + override def getTable(db: String, table: String): CatalogTable = synchronized { requireTableExists(db, table) catalog(db).tables(table).table diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 62d0da076b5a..4f840dd69832 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -235,14 +235,18 @@ class SessionCatalog( * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterTable(tableDefinition: CatalogTable): Unit = { + def alterTable(tableDefinition: CatalogTable, fromAnalyze: Boolean = false): Unit = { val db = formatDatabaseName(tableDefinition.identifier.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableDefinition.identifier.table) val tableIdentifier = TableIdentifier(table, Some(db)) val newTableDefinition = tableDefinition.copy(identifier = tableIdentifier) requireDbExists(db) requireTableExists(tableIdentifier) - externalCatalog.alterTable(newTableDefinition) + if (fromAnalyze) { + externalCatalog.alterTableStats(newTableDefinition) + } else { + externalCatalog.alterTable(newTableDefinition) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 8408d765d491..335de8cfbd7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -22,7 +22,7 @@ import java.util.Date import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.types.StructType @@ -130,6 +130,7 @@ case class CatalogTable( createTime: Long = System.currentTimeMillis, lastAccessTime: Long = -1, properties: Map[String, String] = Map.empty, + catalogStats: Option[Statistics] = None, viewOriginalText: Option[String] = None, viewText: Option[String] = None, comment: Option[String] = None, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index 6e6cc6962c00..d1625962ceba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -31,6 +31,10 @@ package org.apache.spark.sql.catalyst.plans.logical * * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it * defaults to the product of children's `sizeInBytes`. + * @param rowCount Estimated number of rows. * @param isBroadcastable If true, output is small enough to be used in a broadcast join. */ -case class Statistics(sizeInBytes: BigInt, isBroadcastable: Boolean = false) +case class Statistics( + sizeInBytes: BigInt, + rowCount: Option[BigInt] = None, + isBroadcastable: Boolean = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 1eadde223b91..1bc4be034908 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -99,9 +99,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { ctx.identifier.getText.toLowerCase == "noscan") { AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString) } else { - // Always just run the no scan analyze. We should fix this and implement full analyze - // command in the future. - AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString) + AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier).toString, noscan = false) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index a469d4da8613..91090d3fb72b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -24,6 +24,8 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} +import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.execution.datasources.LogicalRelation /** @@ -33,7 +35,7 @@ import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} * Right now, it only supports Hive tables and it only updates the size of a Hive table * in the Hive metastore. */ -case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { +case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { val sessionState = sparkSession.sessionState @@ -71,8 +73,7 @@ case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { size } - val tableParameters = catalogTable.properties - val oldTotalSize = tableParameters.get("totalSize").map(_.toLong).getOrElse(0L) + val oldTotalSize = catalogTable.catalogStats.map(_.sizeInBytes.toLong).getOrElse(0L) val newTotalSize = catalogTable.storage.locationUri.map { p => val path = new Path(p) @@ -88,14 +89,70 @@ case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { } }.getOrElse(0L) - // Update the Hive metastore if the total size of the table is different than the size - // recorded in the Hive metastore. - // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). - if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + var needUpdate = false + val totalSize = if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + needUpdate = true + newTotalSize + } else { + oldTotalSize + } + var numRows: Option[BigInt] = None + if (!noscan) { + val oldRowCount: Long = if (catalogTable.catalogStats.isDefined) { + catalogTable.catalogStats.get.rowCount.map(_.toLong).getOrElse(-1L) + } else { + -1L + } + val newRowCount = sparkSession.table(tableName).count() + if (newRowCount >= 0 && newRowCount != oldRowCount) { + numRows = Some(BigInt(newRowCount)) + needUpdate = true + } + } + // Update the metastore if the above statistics of the table are different from those + // recorded in the metastore. + if (needUpdate) { + sessionState.catalog.alterTable( + catalogTable.copy( + catalogStats = Some(Statistics( + sizeInBytes = totalSize, rowCount = numRows))), + fromAnalyze = true) + + // Refresh the cache of the table in the catalog. + sessionState.catalog.refreshTable(tableIdent) + } + + // data source tables have been converted into LogicalRelations + case logicalRel: LogicalRelation if logicalRel.metastoreTableIdentifier.isDefined => + val oldTotalSize = logicalRel.statistics.sizeInBytes.toLong + val newTotalSize = logicalRel.relation.sizeInBytes + var needUpdate = false + val totalSize = if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + needUpdate = true + newTotalSize + } else { + oldTotalSize + } + var numRows: Option[BigInt] = None + if (!noscan) { + val oldRowCount = logicalRel.statistics.rowCount.map(_.toLong).getOrElse(-1L) + val newRowCount = sparkSession.table(tableName).count() + if (newRowCount >= 0 && newRowCount != oldRowCount) { + numRows = Some(BigInt(newRowCount)) + needUpdate = true + } + } + if (needUpdate) { + val tableIdentifier = logicalRel.metastoreTableIdentifier.get + val catalogTable = sessionState.catalog.getTableMetadata(tableIdentifier) sessionState.catalog.alterTable( catalogTable.copy( - properties = relation.catalogTable.properties + - (AnalyzeTableCommand.TOTAL_SIZE_FIELD -> newTotalSize.toString))) + catalogStats = Some(Statistics( + sizeInBytes = totalSize, rowCount = numRows))), + fromAnalyze = true) + + // Refresh the cache of the table in the catalog. + sessionState.catalog.refreshTable(tableIdentifier) } case otherRelation => @@ -105,7 +162,3 @@ case class AnalyzeTableCommand(tableName: String) extends RunnableCommand { Seq.empty[Row] } } - -object AnalyzeTableCommand { - val TOTAL_SIZE_FIELD = "totalSize" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index a6621054fc74..a25b1e656a78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -207,9 +207,11 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] className = table.provider.get, options = table.storage.properties) - LogicalRelation( + val logicalRel = LogicalRelation( dataSource.resolveRelation(), metastoreTableIdentifier = Some(table.identifier)) + logicalRel.inheritedStats = table.catalogStats + logicalRel } override def apply(plan: LogicalPlan): LogicalPlan = plan transform { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 2a8e147011f5..cf3cdbc0077c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -72,9 +72,12 @@ case class LogicalRelation( // expId can be different but the relation is still the same. override lazy val cleanArgs: Seq[Any] = Seq(relation) - @transient override lazy val statistics: Statistics = Statistics( - sizeInBytes = BigInt(relation.sizeInBytes) - ) + // statistics inherited from a CatalogRelation + @transient var inheritedStats: Option[Statistics] = None + + @transient override lazy val statistics: Statistics = { + inheritedStats.getOrElse(Statistics(sizeInBytes = BigInt(relation.sizeInBytes))) + } /** Used to lookup original attribute capitalization */ val attributeMap: AttributeMap[AttributeReference] = AttributeMap(output.map(o => (o, o))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index ab27381c0600..8fdbd0f2c6da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -192,7 +192,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { * Right now, it only supports catalog tables and it only updates the size of a catalog table * in the external catalog. */ - def analyze(tableName: String): Unit = { - AnalyzeTableCommand(tableName).run(sparkSession) + def analyze(tableName: String, noscan: Boolean = true): Unit = { + AnalyzeTableCommand(tableName, noscan).run(sparkSession) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index 2c81cbf15f08..ae798a3f0645 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.plans.logical.{GlobalLimit, Join, LocalLimit} +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -75,4 +76,31 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { } } + test("test table-level statistics for data source table created in InMemoryCatalog") { + def checkTableStats(tableName: String, totalSize: BigInt, rowCount: Option[BigInt]): Unit = { + val df = sql(s"SELECT * FROM $tableName") + val statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => + rel.statistics + } + assert(statsSeq.size === 1) + assert(statsSeq.head.sizeInBytes === totalSize) + assert(statsSeq.head.rowCount === rowCount) + } + + val tableName = "tbl" + withTable(tableName) { + sql(s"CREATE TABLE $tableName(i INT, j STRING) USING parquet") + Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.mode("overwrite").insertInto("tbl") + + // noscan won't count the number of rows + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") + // sizeInBytes will be updated by analyze command for tables in InMemoryCatalog + checkTableStats(tableName, 984, None) + + // without noscan, we count the number of rows + Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.mode("append").insertInto("tbl") + sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") + checkTableStats(tableName, 1968, Some(4)) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 7f50e38d30c9..fd8f666d31bc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -401,6 +401,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } } + override def alterTableStats(tableDefinition: CatalogTable): Unit = withClient { + assert(tableDefinition.identifier.database.isDefined) + val db = tableDefinition.identifier.database.get + requireTableExists(db, tableDefinition.identifier.table) + client.alterTableStats(tableDefinition) + } + override def getTable(db: String, table: String): CatalogTable = withClient { restoreTableMetadata(client.getTable(db, table)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index ff82c7f7af6f..7ac4349df95f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -80,9 +80,11 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log className = table.provider.get, options = table.storage.properties) - LogicalRelation( + val logicalRel = LogicalRelation( dataSource.resolveRelation(checkPathExist = true), metastoreTableIdentifier = Some(TableIdentifier(in.name, Some(in.database)))) + logicalRel.inheritedStats = table.catalogStats + logicalRel } } @@ -296,7 +298,9 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log logicalRelation } - result.copy(expectedOutputAttributes = Some(metastoreRelation.output)) + val logicalRel = result.copy(expectedOutputAttributes = Some(metastoreRelation.output)) + logicalRel.inheritedStats = Some(metastoreRelation.statistics) + logicalRel } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index d62bc983d027..4363be8f66fd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -108,8 +108,8 @@ private[hive] case class MetastoreRelation( new HiveTable(tTable) } - @transient override lazy val statistics: Statistics = Statistics( - sizeInBytes = { + @transient override lazy val statistics: Statistics = { + val sizeInBytes = { val totalSize = hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE) val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) // TODO: check if this estimate is valid for tables after partition pruning. @@ -140,7 +140,12 @@ private[hive] case class MetastoreRelation( sparkSession.sessionState.conf.defaultSizeInBytes }) } - ) + if (catalogTable.catalogStats.isDefined) { + catalogTable.catalogStats.get.copy(sizeInBytes = sizeInBytes) + } else { + Statistics(sizeInBytes) + } + } // When metastore partition pruning is turned off, we cache the list of all partitions to // mimic the behavior of Spark < 1.5 diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index 6f009d714bef..9c463e7ec037 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.Statistics /** @@ -88,6 +89,9 @@ private[hive] trait HiveClient { /** Updates the given table with new metadata, optionally renaming the table. */ def alterTable(tableName: String, table: CatalogTable): Unit + /** Alter the given table with new statistics. */ + def alterTableStats(table: CatalogTable): Unit + /** Creates a new database with the given name. */ def createDatabase(database: CatalogDatabase, ignoreIfExists: Boolean): Unit diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index b45ad30dcae4..13ff2a7fac05 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -25,6 +25,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} @@ -44,6 +45,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} +import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.{CircularBuffer, Utils} @@ -401,6 +403,7 @@ private[hive] class HiveClientImpl( .map(_.asScala.toMap).orNull ), properties = properties.filter(kv => kv._1 != "comment"), + catalogStats = constructStatsFromHive(properties), comment = properties.get("comment"), viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), @@ -427,6 +430,43 @@ private[hive] class HiveClientImpl( client.alterTable(qualifiedTableName, hiveTable) } + override def alterTableStats(table: CatalogTable): Unit = withHiveState { + // convert Spark's statistics to Hive table's properties + var statsProperties: Map[String, String] = Map() + if (table.catalogStats.isDefined) { + val stats = table.catalogStats.get + statsProperties += (StatsSetupConst.TOTAL_SIZE -> stats.sizeInBytes.toString()) + if (stats.rowCount.isDefined) { + // We need to set STATS_GENERATED_VIA_STATS_TASK here so that we can persist + // ROW_COUNT in metastore. This constraint comes from Hive metastore (HIVE-8648). + statsProperties += (StatsSetupConst.ROW_COUNT -> stats.rowCount.get.toString(), + StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK -> StatsSetupConst.TRUE) + } + } + + val hiveTable = if (statsProperties.nonEmpty) { + toHiveTable(table.copy(properties = table.properties ++ statsProperties)) + } else { + toHiveTable(table) + } + client.alterTable(table.qualifiedName, hiveTable) + } + + // Construct Spark's statistics from information in Hive metastore. + private def constructStatsFromHive(properties: Map[String, String]): Option[Statistics] = { + val statsTrue = properties.getOrElse(StatsSetupConst.COLUMN_STATS_ACCURATE, "false").toBoolean + if (statsTrue && properties.contains(StatsSetupConst.TOTAL_SIZE)) { + val totalSize = BigInt(properties.get(StatsSetupConst.TOTAL_SIZE).get) + // TODO: we will compute "estimatedSize" when we have column stats: + // average size of row * number of rows + Some(Statistics( + sizeInBytes = totalSize, + rowCount = properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)))) + } else { + None + } + } + override def createPartitions( db: String, table: String, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index b275ab17a93c..d6dc25861119 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.command.AnalyzeTableCommand +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf @@ -168,6 +169,88 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils TableIdentifier("tempTable"), ignoreIfNotExists = true, purge = false) } + test("test table-level statistics for hive tables created in HiveExternalCatalog") { + def checkTableStats(tableName: String, rowCount: Option[BigInt]): Unit = { + val df = sql(s"SELECT * FROM $tableName") + val statsSeq = df.queryExecution.analyzed.collect { case mr: MetastoreRelation => + mr.statistics + } + assert(statsSeq.size === 1) + assert(statsSeq.head.rowCount === rowCount) + } + + val textTable = "textTable" + val parquetTable = "parquetTable" + val orcTable = "orcTable" + withTable(textTable, parquetTable, orcTable) { + sql(s"CREATE TABLE $textTable (key STRING, value STRING) STORED AS TEXTFILE") + sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") + + // noscan won't count the number of rows + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") + checkTableStats(textTable, None) + + // without noscan, we count the number of rows + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") + checkTableStats(textTable, Some(500)) + + // After "ALTER TABLE" command (without setting STATS_GENERATED_VIA_STATS_TASK), + // StatsSetupConst.COLUMN_STATS_ACCURATE will be set to false, this logic is in Hive. + sql(s"alter table $textTable set TBLPROPERTIES ('numRows'='2')") + checkTableStats(textTable, None) + + // test statistics of LogicalRelation inherited from MetastoreRelation + sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) STORED AS PARQUET") + sql(s"CREATE TABLE $orcTable (key STRING, value STRING) STORED AS ORC") + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") + sql(s"INSERT INTO TABLE $orcTable SELECT * FROM src") + sql(s"INSERT INTO TABLE $orcTable SELECT * FROM src") + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") + sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") + + var df = sql(s"SELECT * FROM $parquetTable") + var statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => + rel.statistics + } + assert(statsSeq.size === 1) + assert(statsSeq.head.rowCount === Some(500)) + + withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { + df = sql(s"SELECT * FROM $orcTable") + statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => + rel.statistics + } + assert(statsSeq.size === 1) + assert(statsSeq.head.rowCount === Some(1000)) + } + } + } + + test("test table-level statistics for data source table created in HiveExternalCatalog") { + def checkTableStats(tableName: String, rowCount: Option[BigInt]): Unit = { + val df = sql(s"SELECT * FROM $tableName") + val statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => + rel.statistics + } + assert(statsSeq.size === 1) + assert(statsSeq.head.rowCount === rowCount) + } + + val parquetTable = "parquetTable" + withTable(parquetTable) { + sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) USING PARQUET") + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") + + // noscan won't count the number of rows + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") + checkTableStats(parquetTable, None) + + // without noscan, we count the number of rows + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") + checkTableStats(parquetTable, Some(500)) + } + } + test("estimates the size of a test MetastoreRelation") { val df = sql("""SELECT * FROM src""") val sizes = df.queryExecution.analyzed.collect { case mr: MetastoreRelation => From 001ef88e37f3903d317f52a587373f5bd060dbce Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Thu, 25 Aug 2016 19:19:51 +0800 Subject: [PATCH 03/25] fix testcase after rebase --- .../scala/org/apache/spark/sql/StatisticsSuite.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index ae798a3f0645..e2f091fa37f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -77,13 +77,12 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { } test("test table-level statistics for data source table created in InMemoryCatalog") { - def checkTableStats(tableName: String, totalSize: BigInt, rowCount: Option[BigInt]): Unit = { + def checkTableStats(tableName: String, rowCount: Option[BigInt]): Unit = { val df = sql(s"SELECT * FROM $tableName") val statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => rel.statistics } assert(statsSeq.size === 1) - assert(statsSeq.head.sizeInBytes === totalSize) assert(statsSeq.head.rowCount === rowCount) } @@ -94,13 +93,11 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { // noscan won't count the number of rows sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") - // sizeInBytes will be updated by analyze command for tables in InMemoryCatalog - checkTableStats(tableName, 984, None) + checkTableStats(tableName, None) // without noscan, we count the number of rows - Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.mode("append").insertInto("tbl") sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") - checkTableStats(tableName, 1968, Some(4)) + checkTableStats(tableName, Some(2)) } } } From a2387eef05d281ce8ebb2935d78da6f3025d848e Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Thu, 25 Aug 2016 19:21:38 +0800 Subject: [PATCH 04/25] recover path --- .../scala/org/apache/spark/sql/execution/SparkSqlParser.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 1bc4be034908..7099d7b196dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -388,11 +388,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { - val str = ctx.path.getText - val pathStr = str.substring(1, str.length - 1) LoadDataCommand( table = visitTableIdentifier(ctx.tableIdentifier), - path = pathStr, + path = string(ctx.path), isLocal = ctx.LOCAL != null, isOverwrite = ctx.OVERWRITE != null, partition = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) From dad43f70d7c384addadfda9240a030b3971873c5 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Thu, 25 Aug 2016 19:28:57 +0800 Subject: [PATCH 05/25] add comment --- .../scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 13ff2a7fac05..41f46c15e787 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -455,6 +455,7 @@ private[hive] class HiveClientImpl( // Construct Spark's statistics from information in Hive metastore. private def constructStatsFromHive(properties: Map[String, String]): Option[Statistics] = { val statsTrue = properties.getOrElse(StatsSetupConst.COLUMN_STATS_ACCURATE, "false").toBoolean + // If the stats are not accurate, we do not load them. if (statsTrue && properties.contains(StatsSetupConst.TOTAL_SIZE)) { val totalSize = BigInt(properties.get(StatsSetupConst.TOTAL_SIZE).get) // TODO: we will compute "estimatedSize" when we have column stats: From 4b4358f62d08c9ae63588f10368a7155cf9bdfec Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Fri, 26 Aug 2016 10:32:01 +0800 Subject: [PATCH 06/25] use flatmap instead of if-else --- .../sql/execution/command/AnalyzeTableCommand.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index 91090d3fb72b..a5c9d36c0026 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -73,7 +73,8 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend size } - val oldTotalSize = catalogTable.catalogStats.map(_.sizeInBytes.toLong).getOrElse(0L) + val catalogStats = catalogTable.catalogStats + val oldTotalSize = catalogStats.map(_.sizeInBytes.toLong).getOrElse(0L) val newTotalSize = catalogTable.storage.locationUri.map { p => val path = new Path(p) @@ -98,11 +99,7 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend } var numRows: Option[BigInt] = None if (!noscan) { - val oldRowCount: Long = if (catalogTable.catalogStats.isDefined) { - catalogTable.catalogStats.get.rowCount.map(_.toLong).getOrElse(-1L) - } else { - -1L - } + val oldRowCount = catalogStats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L) val newRowCount = sparkSession.table(tableName).count() if (newRowCount >= 0 && newRowCount != oldRowCount) { numRows = Some(BigInt(newRowCount)) From cb6ab95e53b41c040f2a05ec2a24acbfd8e7befe Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Fri, 26 Aug 2016 17:25:11 +0800 Subject: [PATCH 07/25] move alterTableStats logic to HiveExternalCatalog --- .../command/AnalyzeTableCommand.scala | 3 --- .../spark/sql/hive/HiveExternalCatalog.scala | 24 +++++++++++++++---- .../spark/sql/hive/client/HiveClient.scala | 4 ---- .../sql/hive/client/HiveClientImpl.scala | 22 ----------------- 4 files changed, 20 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index a5c9d36c0026..a3d9eeefdd1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -31,9 +31,6 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation /** * Analyzes the given table in the current database to generate statistics, which will be * used in query optimizations. - * - * Right now, it only supports Hive tables and it only updates the size of a Hive table - * in the Hive metastore. */ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extends RunnableCommand { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index fd8f666d31bc..17192b3cc7e9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -23,6 +23,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.thrift.TException @@ -402,10 +403,25 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } override def alterTableStats(tableDefinition: CatalogTable): Unit = withClient { - assert(tableDefinition.identifier.database.isDefined) - val db = tableDefinition.identifier.database.get - requireTableExists(db, tableDefinition.identifier.table) - client.alterTableStats(tableDefinition) + // convert Spark's statistics to Hive table's properties + var statsProperties: Map[String, String] = Map() + if (tableDefinition.catalogStats.isDefined) { + val stats = tableDefinition.catalogStats.get + statsProperties += (StatsSetupConst.TOTAL_SIZE -> stats.sizeInBytes.toString()) + if (stats.rowCount.isDefined) { + // We need to set STATS_GENERATED_VIA_STATS_TASK here so that we can persist + // ROW_COUNT in metastore. This constraint comes from Hive metastore (HIVE-8648). + statsProperties += (StatsSetupConst.ROW_COUNT -> stats.rowCount.get.toString(), + StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK -> StatsSetupConst.TRUE) + } + } + + val table = if (statsProperties.nonEmpty) { + tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties) + } else { + tableDefinition + } + alterTable(table) } override def getTable(db: String, table: String): CatalogTable = withClient { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala index 9c463e7ec037..6f009d714bef 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClient.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.catalyst.plans.logical.Statistics /** @@ -89,9 +88,6 @@ private[hive] trait HiveClient { /** Updates the given table with new metadata, optionally renaming the table. */ def alterTable(tableName: String, table: CatalogTable): Unit - /** Alter the given table with new statistics. */ - def alterTableStats(table: CatalogTable): Unit - /** Creates a new database with the given name. */ def createDatabase(database: CatalogDatabase, ignoreIfExists: Boolean): Unit diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 41f46c15e787..955c8de9acb7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -430,28 +430,6 @@ private[hive] class HiveClientImpl( client.alterTable(qualifiedTableName, hiveTable) } - override def alterTableStats(table: CatalogTable): Unit = withHiveState { - // convert Spark's statistics to Hive table's properties - var statsProperties: Map[String, String] = Map() - if (table.catalogStats.isDefined) { - val stats = table.catalogStats.get - statsProperties += (StatsSetupConst.TOTAL_SIZE -> stats.sizeInBytes.toString()) - if (stats.rowCount.isDefined) { - // We need to set STATS_GENERATED_VIA_STATS_TASK here so that we can persist - // ROW_COUNT in metastore. This constraint comes from Hive metastore (HIVE-8648). - statsProperties += (StatsSetupConst.ROW_COUNT -> stats.rowCount.get.toString(), - StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK -> StatsSetupConst.TRUE) - } - } - - val hiveTable = if (statsProperties.nonEmpty) { - toHiveTable(table.copy(properties = table.properties ++ statsProperties)) - } else { - toHiveTable(table) - } - client.alterTable(table.qualifiedName, hiveTable) - } - // Construct Spark's statistics from information in Hive metastore. private def constructStatsFromHive(properties: Map[String, String]): Option[Statistics] = { val statsTrue = properties.getOrElse(StatsSetupConst.COLUMN_STATS_ACCURATE, "false").toBoolean From 10f6e07ffbaf55d9232132621e5d0aa320212e89 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Sat, 27 Aug 2016 14:46:25 +0800 Subject: [PATCH 08/25] fix test case: set sizeInBytes in statistics of LogicalRelation no matter inheritedStats exists or not --- .../spark/sql/execution/datasources/LogicalRelation.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index cf3cdbc0077c..dec207a850b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -76,7 +76,8 @@ case class LogicalRelation( @transient var inheritedStats: Option[Statistics] = None @transient override lazy val statistics: Statistics = { - inheritedStats.getOrElse(Statistics(sizeInBytes = BigInt(relation.sizeInBytes))) + inheritedStats.map(_.copy(sizeInBytes = relation.sizeInBytes)).getOrElse( + Statistics(sizeInBytes = relation.sizeInBytes)) } /** Used to lookup original attribute capitalization */ From eb8b9c092463512648178287c44747192a505f21 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Mon, 29 Aug 2016 09:55:30 +0800 Subject: [PATCH 09/25] adding inheritedStats as a constructor parameter in LogicalRelation --- .../datasources/DataSourceStrategy.scala | 24 +++++++++---------- .../datasources/LogicalRelation.scala | 12 +++++----- .../sql/execution/datasources/rules.scala | 16 ++++++------- .../datasources/FileSourceStrategySuite.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 15 ++++++------ .../apache/spark/sql/hive/parquetSuites.scala | 8 +++---- 6 files changed, 38 insertions(+), 39 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index a25b1e656a78..81fdac117e09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -144,7 +144,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { // dynamic_partitioning_columns are partitioning columns that do not assigned // values in the PARTITION clause (e.g. c in the above example). case insert @ logical.InsertIntoTable( - relation @ LogicalRelation(t: HadoopFsRelation, _, _), parts, query, overwrite, false) + relation @ LogicalRelation(t: HadoopFsRelation, _, _, _), parts, query, overwrite, false) if query.resolved && parts.exists(_._2.isDefined) => val projectList = convertStaticPartitions( @@ -159,7 +159,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: HadoopFsRelation, _, _), part, query, overwrite, false) + l @ LogicalRelation(t: HadoopFsRelation, _, _, _), part, query, overwrite, false) if query.resolved && t.schema.asNullable == query.schema.asNullable => // Sanity checks @@ -170,7 +170,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { val outputPath = t.location.paths.head val inputPaths = query.collect { - case LogicalRelation(r: HadoopFsRelation, _, _) => r.location.paths + case LogicalRelation(r: HadoopFsRelation, _, _, _) => r.location.paths }.flatten val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append @@ -207,11 +207,10 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] className = table.provider.get, options = table.storage.properties) - val logicalRel = LogicalRelation( + LogicalRelation( dataSource.resolveRelation(), - metastoreTableIdentifier = Some(table.identifier)) - logicalRel.inheritedStats = table.catalogStats - logicalRel + metastoreTableIdentifier = Some(table.identifier), + inheritedStats = table.catalogStats) } override def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -230,7 +229,7 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] */ object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _, _)) => pruneFilterProjectRaw( l, projects, @@ -238,21 +237,22 @@ object DataSourceStrategy extends Strategy with Logging { (requestedColumns, allPredicates, _) => toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, allPredicates))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _, _)) => + case PhysicalOperation(projects, filters, + l @ LogicalRelation(t: PrunedFilteredScan, _, _, _)) => pruneFilterProject( l, projects, filters, (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _, _)) => pruneFilterProject( l, projects, filters, (a, _) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray))) :: Nil - case l @ LogicalRelation(baseRelation: TableScan, _, _) => + case l @ LogicalRelation(baseRelation: TableScan, _, _, _) => RowDataSourceScanExec( l.output, toCatalystRDD(l, baseRelation.buildScan()), @@ -261,7 +261,7 @@ object DataSourceStrategy extends Strategy with Logging { Map.empty, None) :: Nil - case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), + case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _, _), part, query, overwrite, false) if part.isEmpty => ExecutedCommandExec(InsertIntoDataSourceCommand(l, query, overwrite)) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index dec207a850b8..3fb2eb486af8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -33,7 +33,8 @@ import org.apache.spark.util.Utils case class LogicalRelation( relation: BaseRelation, expectedOutputAttributes: Option[Seq[Attribute]] = None, - metastoreTableIdentifier: Option[TableIdentifier] = None) + metastoreTableIdentifier: Option[TableIdentifier] = None, + inheritedStats: Option[Statistics] = None) extends LeafNode with MultiInstanceRelation { override val output: Seq[AttributeReference] = { @@ -52,7 +53,8 @@ case class LogicalRelation( // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any): Boolean = other match { - case l @ LogicalRelation(otherRelation, _, _) => relation == otherRelation && output == l.output + case l @ LogicalRelation(otherRelation, _, _, _) => + relation == otherRelation && output == l.output case _ => false } @@ -62,7 +64,7 @@ case class LogicalRelation( override def sameResult(otherPlan: LogicalPlan): Boolean = { otherPlan.canonicalized match { - case LogicalRelation(otherRelation, _, _) => relation == otherRelation + case LogicalRelation(otherRelation, _, _, _) => relation == otherRelation case _ => false } } @@ -72,9 +74,7 @@ case class LogicalRelation( // expId can be different but the relation is still the same. override lazy val cleanArgs: Seq[Any] = Seq(relation) - // statistics inherited from a CatalogRelation - @transient var inheritedStats: Option[Statistics] = None - + // inheritedStats is inherited from a CatalogRelation @transient override lazy val statistics: Statistics = { inheritedStats.map(_.copy(sizeInBytes = relation.sizeInBytes)).getOrElse( Statistics(sizeInBytes = relation.sizeInBytes)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index f14c63c19f90..829f69983684 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -252,10 +252,10 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { case relation: CatalogRelation => val metadata = relation.catalogTable preprocess(i, metadata.identifier.quotedString, metadata.partitionColumnNames) - case LogicalRelation(h: HadoopFsRelation, _, identifier) => + case LogicalRelation(h: HadoopFsRelation, _, identifier, _) => val tblName = identifier.map(_.quotedString).getOrElse("unknown") preprocess(i, tblName, h.partitionSchema.map(_.name)) - case LogicalRelation(_: InsertableRelation, _, identifier) => + case LogicalRelation(_: InsertableRelation, _, identifier, _) => val tblName = identifier.map(_.quotedString).getOrElse("unknown") preprocess(i, tblName, Nil) case other => i @@ -306,7 +306,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation, _, _), + l @ LogicalRelation(t: InsertableRelation, _, _, _), partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { @@ -314,7 +314,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } else { // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _, _) => src + case LogicalRelation(src: BaseRelation, _, _, _) => src } if (srcRelations.contains(t)) { failAnalysis( @@ -325,7 +325,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } case logical.InsertIntoTable( - LogicalRelation(r: HadoopFsRelation, _, _), part, query, overwrite, _) => + LogicalRelation(r: HadoopFsRelation, _, _, _), part, query, overwrite, _) => // We need to make sure the partition columns specified by users do match partition // columns of the relation. val existingPartitionColumns = r.partitionSchema.fieldNames.toSet @@ -344,7 +344,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _, _) => src + case LogicalRelation(src: BaseRelation, _, _, _) => src } if (srcRelations.contains(r)) { failAnalysis( @@ -365,10 +365,10 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) EliminateSubqueryAliases(catalog.lookupRelation(tableDesc.identifier)) match { // Only do the check if the table is a data source table // (the relation is a BaseRelation). - case l @ LogicalRelation(dest: BaseRelation, _, _) => + case l @ LogicalRelation(dest: BaseRelation, _, _, _) => // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _, _) => src + case LogicalRelation(src: BaseRelation, _, _, _) => src } if (srcRelations.contains(dest)) { failAnalysis( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 09fd75018035..2877754993a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -506,7 +506,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi if (buckets > 0) { val bucketed = df.queryExecution.analyzed transform { - case l @ LogicalRelation(r: HadoopFsRelation, _, _) => + case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => l.copy(relation = r.copy(bucketSpec = Some(BucketSpec(numBuckets = buckets, "c1" :: Nil, Nil)))) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 7ac4349df95f..58693ae3d7fd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -80,11 +80,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log className = table.provider.get, options = table.storage.properties) - val logicalRel = LogicalRelation( + LogicalRelation( dataSource.resolveRelation(checkPathExist = true), - metastoreTableIdentifier = Some(TableIdentifier(in.name, Some(in.database)))) - logicalRel.inheritedStats = table.catalogStats - logicalRel + metastoreTableIdentifier = Some(TableIdentifier(in.name, Some(in.database))), + inheritedStats = table.catalogStats) } } @@ -147,7 +146,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => None // Cache miss - case logical @ LogicalRelation(relation: HadoopFsRelation, _, _) => + case logical @ LogicalRelation(relation: HadoopFsRelation, _, _, _) => val cachedRelationFileFormatClass = relation.fileFormat.getClass expectedFileFormat match { @@ -298,9 +297,9 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log logicalRelation } - val logicalRel = result.copy(expectedOutputAttributes = Some(metastoreRelation.output)) - logicalRel.inheritedStats = Some(metastoreRelation.statistics) - logicalRel + result.copy( + expectedOutputAttributes = Some(metastoreRelation.output), + inheritedStats = Some(metastoreRelation.statistics)) } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index e92bbdea75a7..1d11ccb7faf9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -284,7 +284,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { ) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(_: HadoopFsRelation, _, _) => // OK + case LogicalRelation(_: HadoopFsRelation, _, _, _) => // OK case _ => fail( "test_parquet_ctas should be converted to " + s"${classOf[HadoopFsRelation ].getCanonicalName }") @@ -371,7 +371,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { assertResult(2) { analyzed.collect { - case r @ LogicalRelation(_: HadoopFsRelation, _, _) => r + case r @ LogicalRelation(_: HadoopFsRelation, _, _, _) => r }.size } } @@ -380,7 +380,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { def collectHadoopFsRelation(df: DataFrame): HadoopFsRelation = { val plan = df.queryExecution.analyzed plan.collectFirst { - case LogicalRelation(r: HadoopFsRelation, _, _) => r + case LogicalRelation(r: HadoopFsRelation, _, _, _) => r }.getOrElse { fail(s"Expecting a HadoopFsRelation 2, but got:\n$plan") } @@ -455,7 +455,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { // Converted test_parquet should be cached. sessionState.catalog.getCachedDataSourceTable(tableIdentifier) match { case null => fail("Converted test_parquet should be cached in the cache.") - case logical @ LogicalRelation(parquetRelation: HadoopFsRelation, _, _) => // OK + case logical @ LogicalRelation(parquetRelation: HadoopFsRelation, _, _, _) => // OK case other => fail( "The cached test_parquet should be a Parquet Relation. " + From ac332ecdddde46b6d13f9aa14cc0f197d84b082a Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Mon, 29 Aug 2016 10:06:37 +0800 Subject: [PATCH 10/25] Revert "recover path" This reverts commit 9589a172f907792a3567482b5482010be7dbd906. --- .../scala/org/apache/spark/sql/execution/SparkSqlParser.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 7099d7b196dd..1bc4be034908 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -388,9 +388,11 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { + val str = ctx.path.getText + val pathStr = str.substring(1, str.length - 1) LoadDataCommand( table = visitTableIdentifier(ctx.tableIdentifier), - path = string(ctx.path), + path = pathStr, isLocal = ctx.LOCAL != null, isOverwrite = ctx.OVERWRITE != null, partition = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) From 0fb2149d941a5a1fd72abc2ca41adee9718df24c Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Mon, 29 Aug 2016 11:20:33 +0800 Subject: [PATCH 11/25] remove duplicated code --- .../command/AnalyzeTableCommand.scala | 107 ++++++++---------- 1 file changed, 47 insertions(+), 60 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index a3d9eeefdd1e..2cf5a450feba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -39,6 +39,41 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) + def updateTableStats( + catalogTable: CatalogTable, + oldTotalSize: Long, + oldRowCount: Long, + newTotalSize: Long): Unit = { + + var needUpdate = false + val totalSize = if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + needUpdate = true + newTotalSize + } else { + oldTotalSize + } + var numRows: Option[BigInt] = None + if (!noscan) { + val newRowCount = sparkSession.table(tableName).count() + if (newRowCount >= 0 && newRowCount != oldRowCount) { + numRows = Some(BigInt(newRowCount)) + needUpdate = true + } + } + // Update the metastore if the above statistics of the table are different from those + // recorded in the metastore. + if (needUpdate) { + sessionState.catalog.alterTable( + catalogTable.copy( + catalogStats = Some(Statistics( + sizeInBytes = totalSize, rowCount = numRows))), + fromAnalyze = true) + + // Refresh the cache of the table in the catalog. + sessionState.catalog.refreshTable(tableIdent) + } + } + relation match { case relation: CatalogRelation => val catalogTable: CatalogTable = relation.catalogTable @@ -70,8 +105,6 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend size } - val catalogStats = catalogTable.catalogStats - val oldTotalSize = catalogStats.map(_.sizeInBytes.toLong).getOrElse(0L) val newTotalSize = catalogTable.storage.locationUri.map { p => val path = new Path(p) @@ -87,67 +120,21 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend } }.getOrElse(0L) - var needUpdate = false - val totalSize = if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - needUpdate = true - newTotalSize - } else { - oldTotalSize - } - var numRows: Option[BigInt] = None - if (!noscan) { - val oldRowCount = catalogStats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L) - val newRowCount = sparkSession.table(tableName).count() - if (newRowCount >= 0 && newRowCount != oldRowCount) { - numRows = Some(BigInt(newRowCount)) - needUpdate = true - } - } - // Update the metastore if the above statistics of the table are different from those - // recorded in the metastore. - if (needUpdate) { - sessionState.catalog.alterTable( - catalogTable.copy( - catalogStats = Some(Statistics( - sizeInBytes = totalSize, rowCount = numRows))), - fromAnalyze = true) - - // Refresh the cache of the table in the catalog. - sessionState.catalog.refreshTable(tableIdent) - } + updateTableStats( + catalogTable, + oldTotalSize = catalogTable.catalogStats.map(_.sizeInBytes.toLong).getOrElse(0L), + oldRowCount = catalogTable.catalogStats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L), + newTotalSize = newTotalSize) // data source tables have been converted into LogicalRelations case logicalRel: LogicalRelation if logicalRel.metastoreTableIdentifier.isDefined => - val oldTotalSize = logicalRel.statistics.sizeInBytes.toLong - val newTotalSize = logicalRel.relation.sizeInBytes - var needUpdate = false - val totalSize = if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - needUpdate = true - newTotalSize - } else { - oldTotalSize - } - var numRows: Option[BigInt] = None - if (!noscan) { - val oldRowCount = logicalRel.statistics.rowCount.map(_.toLong).getOrElse(-1L) - val newRowCount = sparkSession.table(tableName).count() - if (newRowCount >= 0 && newRowCount != oldRowCount) { - numRows = Some(BigInt(newRowCount)) - needUpdate = true - } - } - if (needUpdate) { - val tableIdentifier = logicalRel.metastoreTableIdentifier.get - val catalogTable = sessionState.catalog.getTableMetadata(tableIdentifier) - sessionState.catalog.alterTable( - catalogTable.copy( - catalogStats = Some(Statistics( - sizeInBytes = totalSize, rowCount = numRows))), - fromAnalyze = true) - - // Refresh the cache of the table in the catalog. - sessionState.catalog.refreshTable(tableIdentifier) - } + val tableIdentifier = logicalRel.metastoreTableIdentifier.get + val catalogTable = sessionState.catalog.getTableMetadata(tableIdentifier) + updateTableStats( + catalogTable, + oldTotalSize = logicalRel.statistics.sizeInBytes.toLong, + oldRowCount = logicalRel.statistics.rowCount.map(_.toLong).getOrElse(-1L), + newTotalSize = logicalRel.relation.sizeInBytes) case otherRelation => throw new AnalysisException(s"ANALYZE TABLE is only supported for Hive tables, " + From bb0861e9a711d3561ffb3b8d66b1ee1bd395793d Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Mon, 29 Aug 2016 11:52:50 +0800 Subject: [PATCH 12/25] simplify code --- .../command/AnalyzeTableCommand.scala | 28 ++++++++----------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index 2cf5a450feba..14f069335b06 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -21,7 +21,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} +import org.apache.spark.sql.{AnalysisException, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogTable} import org.apache.spark.sql.catalyst.plans.logical.Statistics @@ -45,29 +45,25 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend oldRowCount: Long, newTotalSize: Long): Unit = { - var needUpdate = false - val totalSize = if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - needUpdate = true - newTotalSize - } else { - oldTotalSize + var newStats: Option[Statistics] = None + if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + newStats = Some(Statistics(sizeInBytes = newTotalSize)) } - var numRows: Option[BigInt] = None if (!noscan) { - val newRowCount = sparkSession.table(tableName).count() + val newRowCount = Dataset.ofRows(sparkSession, relation).count() if (newRowCount >= 0 && newRowCount != oldRowCount) { - numRows = Some(BigInt(newRowCount)) - needUpdate = true + newStats = if (newStats.isDefined) { + newStats.map(_.copy(rowCount = Some(BigInt(newRowCount)))) + } else { + Some(Statistics(sizeInBytes = oldTotalSize, rowCount = Some(BigInt(newRowCount)))) + } } } // Update the metastore if the above statistics of the table are different from those // recorded in the metastore. - if (needUpdate) { + if (newStats.isDefined) { sessionState.catalog.alterTable( - catalogTable.copy( - catalogStats = Some(Statistics( - sizeInBytes = totalSize, rowCount = numRows))), - fromAnalyze = true) + catalogTable.copy(catalogStats = newStats), fromAnalyze = true) // Refresh the cache of the table in the catalog. sessionState.catalog.refreshTable(tableIdent) From 61f6c27cfeaf240126cef222282304d2d3fe307e Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Mon, 29 Aug 2016 17:23:36 +0800 Subject: [PATCH 13/25] create spark's own names for statistic properties --- .../spark/sql/hive/HiveExternalCatalog.scala | 31 ++++------ .../spark/sql/hive/MetastoreRelation.scala | 5 ++ .../sql/hive/client/HiveClientImpl.scala | 10 ++- .../spark/sql/hive/StatisticsSuite.scala | 62 +++++++------------ 4 files changed, 44 insertions(+), 64 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 17192b3cc7e9..0bba6fc8571a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -23,7 +23,6 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.thrift.TException @@ -403,25 +402,15 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } override def alterTableStats(tableDefinition: CatalogTable): Unit = withClient { - // convert Spark's statistics to Hive table's properties - var statsProperties: Map[String, String] = Map() - if (tableDefinition.catalogStats.isDefined) { - val stats = tableDefinition.catalogStats.get - statsProperties += (StatsSetupConst.TOTAL_SIZE -> stats.sizeInBytes.toString()) - if (stats.rowCount.isDefined) { - // We need to set STATS_GENERATED_VIA_STATS_TASK here so that we can persist - // ROW_COUNT in metastore. This constraint comes from Hive metastore (HIVE-8648). - statsProperties += (StatsSetupConst.ROW_COUNT -> stats.rowCount.get.toString(), - StatsSetupConst.STATS_GENERATED_VIA_STATS_TASK -> StatsSetupConst.TRUE) - } + assert(tableDefinition.catalogStats.isDefined) + val stats = tableDefinition.catalogStats.get + // convert table statistics to properties so that we can persist them through hive api + 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 table = if (statsProperties.nonEmpty) { - tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties) - } else { - tableDefinition - } - alterTable(table) + alterTable(tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties)) } override def getTable(db: String, table: String): CatalogTable = withClient { @@ -633,6 +622,10 @@ object HiveExternalCatalog { val DATASOURCE_SCHEMA_BUCKETCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "bucketCol." val DATASOURCE_SCHEMA_SORTCOL_PREFIX = DATASOURCE_SCHEMA_PREFIX + "sortCol." + val STATISTICS_PREFIX = "spark.sql.statistics." + val STATISTICS_TOTAL_SIZE = STATISTICS_PREFIX + "totalSize" + val STATISTICS_NUM_ROWS = STATISTICS_PREFIX + "numRows" + def getProviderFromTableProperties(metadata: CatalogTable): Option[String] = { metadata.properties.get(DATASOURCE_PROVIDER) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index 4363be8f66fd..c69db59da595 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -112,6 +112,7 @@ private[hive] case class MetastoreRelation( val sizeInBytes = { val totalSize = hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE) val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) + val analyzedSize = hiveQlTable.getParameters.get(HiveExternalCatalog.STATISTICS_TOTAL_SIZE) // 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. @@ -120,12 +121,16 @@ private[hive] case class MetastoreRelation( BigInt( // 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, use `HiveExternalCatalog.STATISTICS_TOTAL_SIZE`, + // which is generated by analyze command. // if the size is still less than zero, we try to get the file size from HDFS. // given this is only needed for optimization, if the HDFS call fails we return the default. if (totalSize != null && totalSize.toLong > 0L) { totalSize.toLong } else if (rawDataSize != null && rawDataSize.toLong > 0) { rawDataSize.toLong + } else if (analyzedSize != null && analyzedSize.toLong > 0) { + analyzedSize.toLong } else if (sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled) { try { val hadoopConf = sparkSession.sessionState.newHadoopConf() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 955c8de9acb7..d9258b388003 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -25,7 +25,6 @@ import scala.language.reflectiveCalls import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema} @@ -47,6 +46,7 @@ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.QueryExecutionException +import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.{CircularBuffer, Utils} @@ -432,15 +432,13 @@ private[hive] class HiveClientImpl( // Construct Spark's statistics from information in Hive metastore. private def constructStatsFromHive(properties: Map[String, String]): Option[Statistics] = { - val statsTrue = properties.getOrElse(StatsSetupConst.COLUMN_STATS_ACCURATE, "false").toBoolean - // If the stats are not accurate, we do not load them. - if (statsTrue && properties.contains(StatsSetupConst.TOTAL_SIZE)) { - val totalSize = BigInt(properties.get(StatsSetupConst.TOTAL_SIZE).get) + if (properties.contains(HiveExternalCatalog.STATISTICS_TOTAL_SIZE)) { + val totalSize = BigInt(properties.get(HiveExternalCatalog.STATISTICS_TOTAL_SIZE).get) // TODO: we will compute "estimatedSize" when we have column stats: // average size of row * number of rows Some(Statistics( sizeInBytes = totalSize, - rowCount = properties.get(StatsSetupConst.ROW_COUNT).map(BigInt(_)))) + rowCount = properties.get(HiveExternalCatalog.STATISTICS_NUM_ROWS).map(BigInt(_)))) } else { None } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index d6dc25861119..64f86b621dbb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins._ @@ -169,16 +170,20 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils TableIdentifier("tempTable"), ignoreIfNotExists = true, purge = false) } - test("test table-level statistics for hive tables created in HiveExternalCatalog") { - def checkTableStats(tableName: String, rowCount: Option[BigInt]): Unit = { - val df = sql(s"SELECT * FROM $tableName") - val statsSeq = df.queryExecution.analyzed.collect { case mr: MetastoreRelation => - mr.statistics - } - assert(statsSeq.size === 1) - assert(statsSeq.head.rowCount === rowCount) + private def checkTableStats( + tableName: String, + totalSize: Long, + rowCount: Option[BigInt]): Unit = { + val df = sql(s"SELECT * FROM $tableName") + val statsSeq = df.queryExecution.analyzed.collect { case rel: LeafNode => + rel.statistics } + assert(statsSeq.size === 1) + assert(statsSeq.head.sizeInBytes === totalSize) + assert(statsSeq.head.rowCount === rowCount) + } + test("test table-level statistics for hive tables created in HiveExternalCatalog") { val textTable = "textTable" val parquetTable = "parquetTable" val orcTable = "orcTable" @@ -188,54 +193,33 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils // noscan won't count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") - checkTableStats(textTable, None) + checkTableStats(textTable, 5812, None) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") - checkTableStats(textTable, Some(500)) - - // After "ALTER TABLE" command (without setting STATS_GENERATED_VIA_STATS_TASK), - // StatsSetupConst.COLUMN_STATS_ACCURATE will be set to false, this logic is in Hive. - sql(s"alter table $textTable set TBLPROPERTIES ('numRows'='2')") - checkTableStats(textTable, None) + checkTableStats(textTable, 5812, Some(500)) // test statistics of LogicalRelation inherited from MetastoreRelation sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) STORED AS PARQUET") sql(s"CREATE TABLE $orcTable (key STRING, value STRING) STORED AS ORC") sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") sql(s"INSERT INTO TABLE $orcTable SELECT * FROM src") - sql(s"INSERT INTO TABLE $orcTable SELECT * FROM src") sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") - var df = sql(s"SELECT * FROM $parquetTable") - var statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => - rel.statistics - } - assert(statsSeq.size === 1) - assert(statsSeq.head.rowCount === Some(500)) + checkTableStats(parquetTable, 4236, Some(500)) + + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") + checkTableStats(parquetTable, 8472, Some(1000)) withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { - df = sql(s"SELECT * FROM $orcTable") - statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => - rel.statistics - } - assert(statsSeq.size === 1) - assert(statsSeq.head.rowCount === Some(1000)) + checkTableStats(orcTable, 3023, Some(500)) } } } test("test table-level statistics for data source table created in HiveExternalCatalog") { - def checkTableStats(tableName: String, rowCount: Option[BigInt]): Unit = { - val df = sql(s"SELECT * FROM $tableName") - val statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => - rel.statistics - } - assert(statsSeq.size === 1) - assert(statsSeq.head.rowCount === rowCount) - } - val parquetTable = "parquetTable" withTable(parquetTable) { sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) USING PARQUET") @@ -243,11 +227,11 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils // noscan won't count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") - checkTableStats(parquetTable, None) + checkTableStats(parquetTable, 4236, None) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") - checkTableStats(parquetTable, Some(500)) + checkTableStats(parquetTable, 4236, Some(500)) } } From 6d6e48234666448e59d5e9e908c9739593249edb Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Mon, 29 Aug 2016 20:04:08 +0800 Subject: [PATCH 14/25] need to clear the old (i.e. inaccurate) stats before alter stats --- .../apache/spark/sql/hive/HiveExternalCatalog.scala | 10 ++++++---- .../org/apache/spark/sql/hive/StatisticsSuite.scala | 4 ++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 0bba6fc8571a..4cdede872158 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -403,14 +403,16 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat override def alterTableStats(tableDefinition: CatalogTable): Unit = withClient { assert(tableDefinition.catalogStats.isDefined) + // first we need to clear the old (i.e. inaccurate) stats + var propertiesWithStats: Map[String, String] = + tableDefinition.properties - (STATISTICS_TOTAL_SIZE, STATISTICS_NUM_ROWS) val stats = tableDefinition.catalogStats.get // convert table statistics to properties so that we can persist them through hive api - var statsProperties: Map[String, String] = - Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) + propertiesWithStats += (STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) if (stats.rowCount.isDefined) { - statsProperties += (STATISTICS_NUM_ROWS -> stats.rowCount.get.toString()) + propertiesWithStats += (STATISTICS_NUM_ROWS -> stats.rowCount.get.toString()) } - alterTable(tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties)) + alterTable(tableDefinition.copy(properties = propertiesWithStats)) } override def getTable(db: String, table: String): CatalogTable = withClient { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 64f86b621dbb..8d494e42c5df 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -210,8 +210,8 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils checkTableStats(parquetTable, 4236, Some(500)) sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") - sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") - checkTableStats(parquetTable, 8472, Some(1000)) + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") + checkTableStats(parquetTable, 8472, None) withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { checkTableStats(orcTable, 3023, Some(500)) From 2db1fbf7ed980747177435290d1cad25ab145aec Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Mon, 29 Aug 2016 20:05:47 +0800 Subject: [PATCH 15/25] Revert "fix path" This reverts commit aeca9ba4b4bedc8f5a59869cb72123a15d8c4921. --- .../org/apache/spark/sql/execution/SparkSqlParser.scala | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 1bc4be034908..295c018f6e11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -388,11 +388,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitLoadData(ctx: LoadDataContext): LogicalPlan = withOrigin(ctx) { - val str = ctx.path.getText - val pathStr = str.substring(1, str.length - 1) LoadDataCommand( table = visitTableIdentifier(ctx.tableIdentifier), - path = pathStr, + path = string(ctx.path), isLocal = ctx.LOCAL != null, isOverwrite = ctx.OVERWRITE != null, partition = Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec) @@ -829,9 +827,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * Create location string. */ override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) { -// string(ctx.STRING) - val str = ctx.STRING().getText - str.substring(1, str.length - 1) + string(ctx.STRING) } /** From 9c27071c05da5f285726381dff7eff3dfab7eda9 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Tue, 30 Aug 2016 10:27:15 +0800 Subject: [PATCH 16/25] fix compilation error for LogicalRelation --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 2 +- .../main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala | 2 +- .../spark/sql/execution/OptimizeMetadataOnlyQuery.scala | 4 ++-- .../spark/sql/execution/command/createDataSourceTables.scala | 2 +- .../spark/sql/execution/datasources/FileSourceStrategy.scala | 2 +- .../execution/datasources/parquet/ParquetFilterSuite.scala | 3 ++- .../datasources/parquet/ParquetPartitionDiscoverySuite.scala | 2 +- .../org/apache/spark/sql/sources/FilteredScanSuite.scala | 2 +- .../org/apache/spark/sql/streaming/FileStreamSinkSuite.scala | 2 +- .../org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../org/apache/spark/sql/hive/execution/SQLQuerySuite.scala | 2 +- .../scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala | 4 ++-- 12 files changed, 15 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 6da99ce0dd68..b9f9f4b17c6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2531,7 +2531,7 @@ class Dataset[T] private[sql]( */ def inputFiles: Array[String] = { val files: Seq[String] = logicalPlan.collect { - case LogicalRelation(fsBasedRelation: FileRelation, _, _) => + case LogicalRelation(fsBasedRelation: FileRelation, _, _, _) => fsBasedRelation.inputFiles case fr: FileRelation => fr.inputFiles diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index dde91b0a8606..de8ca4e9fa49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -590,7 +590,7 @@ class SQLBuilder private ( object ExtractSQLTable { def unapply(plan: LogicalPlan): Option[SQLTable] = plan match { - case l @ LogicalRelation(_, _, Some(TableIdentifier(table, Some(database)))) => + case l @ LogicalRelation(_, _, Some(TableIdentifier(table, Some(database))), _) => Some(SQLTable(database, table, l.output.map(_.withQualifier(None)))) case relation: CatalogRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index 1b7fedca8484..f244d23773a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -95,7 +95,7 @@ case class OptimizeMetadataOnlyQuery( child transform { case plan if plan eq relation => relation match { - case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) => + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) val partitionData = fsRelation.location.listFiles(filters = Nil) LocalRelation(partAttrs, partitionData.map(_.values)) @@ -127,7 +127,7 @@ case class OptimizeMetadataOnlyQuery( object PartitionedRelation { def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { - case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) if fsRelation.partitionSchema.nonEmpty => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) Some(AttributeSet(partAttrs), l) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 7400a0e7bb1f..4be8a10a8a43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -174,7 +174,7 @@ case class CreateDataSourceTableAsSelectCommand( EliminateSubqueryAliases( sessionState.catalog.lookupRelation(tableIdent)) match { - case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => + case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _, _) => // check if the file formats match l.relation match { case r: HadoopFsRelation if r.fileFormat.getClass != dataSource.providingClass => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 8b36caf6f1e0..7ad7a249e7d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -52,7 +52,7 @@ import org.apache.spark.sql.execution.SparkPlan object FileSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projects, filters, - l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table)) => + l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _)) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: // - partition keys only - used to prune directories to read diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 4246b54c21f0..ac1165c8adfa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -64,7 +64,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(relation: HadoopFsRelation, _, _)) => + case PhysicalOperation(_, filters, LogicalRelation(relation: HadoopFsRelation, _, _, _) + ) => maybeRelation = Some(relation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 8d18be9300f7..5e4b420d0da5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -626,7 +626,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: HadoopFsRelation, _, _) => + case LogicalRelation(relation: HadoopFsRelation, _, _, _) => assert(relation.partitionSpec === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index be56c964a18f..461e4fb85c55 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -324,7 +324,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic val table = spark.table("oneToTenFiltered") val relation = table.queryExecution.logical.collectFirst { - case LogicalRelation(r, _, _) => r + case LogicalRelation(r, _, _, _) => r }.get assert( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 19c89f5c4100..fdc9a7b57c16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -182,7 +182,7 @@ class FileStreamSinkSuite extends StreamTest { // Verify that MetadataLogFileCatalog is being used and the correct partitioning schema has // been inferred val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { - case LogicalRelation(baseRelation, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => + case LogicalRelation(baseRelation, _, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => baseRelation.asInstanceOf[HadoopFsRelation] } assert(hadoopdFsRelations.size === 1) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index d77bb5cf95f6..acaed4ec191c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -572,7 +572,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row(3) :: Row(4) :: Nil) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(p: HadoopFsRelation, _, _) => // OK + case LogicalRelation(p: HadoopFsRelation, _, _, _) => // OK case _ => fail(s"test_parquet_ctas should have be converted to ${classOf[HadoopFsRelation]}") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 4ca882f840a5..839028ad8543 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -424,7 +424,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val catalogTable = sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) relation match { - case LogicalRelation(r: HadoopFsRelation, _, _) => + case LogicalRelation(r: HadoopFsRelation, _, _, _) => if (!isDataSourceParquet) { fail( s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala index 471192a369f4..89ffe1442ddc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala @@ -45,7 +45,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) @@ -89,7 +89,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) From 7e39a86030e45f10ae0c171a475c054b7c208d20 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Tue, 30 Aug 2016 11:49:04 +0800 Subject: [PATCH 17/25] fix scala style --- .../org/apache/spark/sql/streaming/FileStreamSinkSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index fdc9a7b57c16..b36656853da0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -182,7 +182,8 @@ class FileStreamSinkSuite extends StreamTest { // Verify that MetadataLogFileCatalog is being used and the correct partitioning schema has // been inferred val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { - case LogicalRelation(baseRelation, _, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => + case LogicalRelation(baseRelation, _, _, _) + if baseRelation.isInstanceOf[HadoopFsRelation] => baseRelation.asInstanceOf[HadoopFsRelation] } assert(hadoopdFsRelations.size === 1) From aef78d4bffac4dad99d1646c659e626a4eccb26b Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Tue, 30 Aug 2016 16:40:06 +0800 Subject: [PATCH 18/25] fix test case --- .../apache/spark/sql/StatisticsSuite.scala | 8 ++-- .../spark/sql/hive/StatisticsSuite.scala | 41 +++++++++++-------- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index e2f091fa37f5..60588c52a44b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -79,11 +79,11 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { test("test table-level statistics for data source table created in InMemoryCatalog") { def checkTableStats(tableName: String, rowCount: Option[BigInt]): Unit = { val df = sql(s"SELECT * FROM $tableName") - val statsSeq = df.queryExecution.analyzed.collect { case rel: LogicalRelation => - rel.statistics + val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation => + assert(rel.statistics.sizeInBytes === rel.relation.sizeInBytes) + assert(rel.statistics.rowCount === rowCount) } - assert(statsSeq.size === 1) - assert(statsSeq.head.rowCount === rowCount) + assert(relations.size === 1) } val tableName = "tbl" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 8d494e42c5df..438f9be56ddb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -23,7 +23,6 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.execution.command.AnalyzeTableCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins._ @@ -170,17 +169,26 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils TableIdentifier("tempTable"), ignoreIfNotExists = true, purge = false) } - private def checkTableStats( + private def checkMetastoreRelationStats( tableName: String, totalSize: Long, rowCount: Option[BigInt]): Unit = { val df = sql(s"SELECT * FROM $tableName") - val statsSeq = df.queryExecution.analyzed.collect { case rel: LeafNode => + val relations = df.queryExecution.analyzed.collect { case rel: MetastoreRelation => rel.statistics + assert(rel.statistics.sizeInBytes === totalSize) + assert(rel.statistics.rowCount === rowCount) } - assert(statsSeq.size === 1) - assert(statsSeq.head.sizeInBytes === totalSize) - assert(statsSeq.head.rowCount === rowCount) + assert(relations.size === 1) + } + + private def checkLogicalRelationStats(tableName: String, rowCount: Option[BigInt]): Unit = { + val df = sql(s"SELECT * FROM $tableName") + val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation => + assert(rel.statistics.sizeInBytes === rel.relation.sizeInBytes) + assert(rel.statistics.rowCount === rowCount) + } + assert(relations.size === 1) } test("test table-level statistics for hive tables created in HiveExternalCatalog") { @@ -193,11 +201,16 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils // noscan won't count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") - checkTableStats(textTable, 5812, None) + checkMetastoreRelationStats(textTable, 5812, None) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") - checkTableStats(textTable, 5812, Some(500)) + checkMetastoreRelationStats(textTable, 5812, Some(500)) + + // test whether the old stats are removed + sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") + checkMetastoreRelationStats(textTable, 11624, None) // test statistics of LogicalRelation inherited from MetastoreRelation sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) STORED AS PARQUET") @@ -207,14 +220,10 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") - checkTableStats(parquetTable, 4236, Some(500)) - - sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") - sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") - checkTableStats(parquetTable, 8472, None) + checkLogicalRelationStats(parquetTable, Some(500)) withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { - checkTableStats(orcTable, 3023, Some(500)) + checkLogicalRelationStats(orcTable, Some(500)) } } } @@ -227,11 +236,11 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils // noscan won't count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") - checkTableStats(parquetTable, 4236, None) + checkLogicalRelationStats(parquetTable, None) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") - checkTableStats(parquetTable, 4236, Some(500)) + checkLogicalRelationStats(parquetTable, Some(500)) } } From c7cc55fe2a42f5af3ffd2c9a81f58a608b636320 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Wed, 31 Aug 2016 16:52:05 +0800 Subject: [PATCH 19/25] update based on comments --- .../catalyst/catalog/ExternalCatalog.scala | 2 - .../catalyst/catalog/InMemoryCatalog.scala | 2 - .../sql/catalyst/catalog/SessionCatalog.scala | 8 +- .../command/AnalyzeTableCommand.scala | 64 +++++++-------- .../apache/spark/sql/StatisticsSuite.scala | 1 + .../spark/sql/hive/HiveExternalCatalog.scala | 39 ++++----- .../spark/sql/hive/MetastoreRelation.scala | 74 ++++++++--------- .../spark/sql/hive/StatisticsSuite.scala | 80 ++++++++++++------- 8 files changed, 140 insertions(+), 130 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala index 8c53dd926ed2..27e1810814c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala @@ -85,8 +85,6 @@ abstract class ExternalCatalog { */ def alterTable(tableDefinition: CatalogTable): Unit - def alterTableStats(tableDefinition: CatalogTable): Unit - def getTable(db: String, table: String): CatalogTable def getTableOption(db: String, table: String): Option[CatalogTable] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 0b6fc5963a34..b55ddcb54b45 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -278,8 +278,6 @@ class InMemoryCatalog( catalog(db).tables(tableDefinition.identifier.table).table = tableDefinition } - override def alterTableStats(tableDefinition: CatalogTable): Unit = alterTable(tableDefinition) - override def getTable(db: String, table: String): CatalogTable = synchronized { requireTableExists(db, table) catalog(db).tables(table).table diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 4f840dd69832..62d0da076b5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -235,18 +235,14 @@ class SessionCatalog( * Note: If the underlying implementation does not support altering a certain field, * this becomes a no-op. */ - def alterTable(tableDefinition: CatalogTable, fromAnalyze: Boolean = false): Unit = { + def alterTable(tableDefinition: CatalogTable): Unit = { val db = formatDatabaseName(tableDefinition.identifier.database.getOrElse(getCurrentDatabase)) val table = formatTableName(tableDefinition.identifier.table) val tableIdentifier = TableIdentifier(table, Some(db)) val newTableDefinition = tableDefinition.copy(identifier = tableIdentifier) requireDbExists(db) requireTableExists(tableIdentifier) - if (fromAnalyze) { - externalCatalog.alterTableStats(newTableDefinition) - } else { - externalCatalog.alterTable(newTableDefinition) - } + externalCatalog.alterTable(newTableDefinition) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index 14f069335b06..c3bdd52f7a9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -39,37 +39,6 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName) val relation = EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent)) - def updateTableStats( - catalogTable: CatalogTable, - oldTotalSize: Long, - oldRowCount: Long, - newTotalSize: Long): Unit = { - - var newStats: Option[Statistics] = None - if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - newStats = Some(Statistics(sizeInBytes = newTotalSize)) - } - if (!noscan) { - val newRowCount = Dataset.ofRows(sparkSession, relation).count() - if (newRowCount >= 0 && newRowCount != oldRowCount) { - newStats = if (newStats.isDefined) { - newStats.map(_.copy(rowCount = Some(BigInt(newRowCount)))) - } else { - Some(Statistics(sizeInBytes = oldTotalSize, rowCount = Some(BigInt(newRowCount)))) - } - } - } - // Update the metastore if the above statistics of the table are different from those - // recorded in the metastore. - if (newStats.isDefined) { - sessionState.catalog.alterTable( - catalogTable.copy(catalogStats = newStats), fromAnalyze = true) - - // Refresh the cache of the table in the catalog. - sessionState.catalog.refreshTable(tableIdent) - } - } - relation match { case relation: CatalogRelation => val catalogTable: CatalogTable = relation.catalogTable @@ -133,9 +102,38 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend newTotalSize = logicalRel.relation.sizeInBytes) case otherRelation => - throw new AnalysisException(s"ANALYZE TABLE is only supported for Hive tables, " + - s"but '${tableIdent.unquotedString}' is a ${otherRelation.nodeName}.") + throw new AnalysisException(s"ANALYZE TABLE is not supported for " + + s"${otherRelation.nodeName}.") } + + def updateTableStats( + catalogTable: CatalogTable, + oldTotalSize: Long, + oldRowCount: Long, + newTotalSize: Long): Unit = { + var newStats: Option[Statistics] = None + if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + newStats = Some(Statistics(sizeInBytes = newTotalSize)) + } + if (!noscan) { + val newRowCount = Dataset.ofRows(sparkSession, relation).count() + if (newRowCount >= 0 && newRowCount != oldRowCount) { + newStats = if (newStats.isDefined) { + newStats.map(_.copy(rowCount = Some(BigInt(newRowCount)))) + } else { + Some(Statistics(sizeInBytes = oldTotalSize, rowCount = Some(BigInt(newRowCount)))) + } + } + } + // Update the metastore if the above statistics of the table are different from those + // recorded in the metastore. + if (newStats.isDefined) { + sessionState.catalog.alterTable(catalogTable.copy(catalogStats = newStats)) + // Refresh the cached data source table in the catalog. + sessionState.catalog.refreshTable(tableIdent) + } + } + Seq.empty[Row] } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index 60588c52a44b..ee30d186c691 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -82,6 +82,7 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation => assert(rel.statistics.sizeInBytes === rel.relation.sizeInBytes) assert(rel.statistics.rowCount === rowCount) + rel } assert(relations.size === 1) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index 4cdede872158..bc609759ff1a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -383,36 +383,37 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat requireTableExists(db, tableDefinition.identifier.table) verifyTableProperties(tableDefinition) - if (DDLUtils.isDatasourceTable(tableDefinition)) { - val oldDef = client.getTable(db, tableDefinition.identifier.table) + // convert table statistics to properties so that we can persist them through hive api + val catalogTable = if (tableDefinition.catalogStats.isDefined) { + // first we need to clear the old (i.e. inaccurate) stats + var propertiesWithStats: Map[String, String] = + tableDefinition.properties -(STATISTICS_TOTAL_SIZE, STATISTICS_NUM_ROWS) + val stats = tableDefinition.catalogStats.get + propertiesWithStats += (STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) + if (stats.rowCount.isDefined) { + propertiesWithStats += (STATISTICS_NUM_ROWS -> stats.rowCount.get.toString()) + } + tableDefinition.copy(properties = propertiesWithStats) + } else { + tableDefinition + } + + if (DDLUtils.isDatasourceTable(catalogTable)) { + val oldDef = client.getTable(db, catalogTable.identifier.table) // 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 = oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) - val newDef = tableDefinition.copy( + val newDef = catalogTable.copy( schema = oldDef.schema, partitionColumnNames = oldDef.partitionColumnNames, bucketSpec = oldDef.bucketSpec, - properties = oldDataSourceProps ++ tableDefinition.properties) + properties = oldDataSourceProps ++ catalogTable.properties) client.alterTable(newDef) } else { - client.alterTable(tableDefinition) - } - } - - override def alterTableStats(tableDefinition: CatalogTable): Unit = withClient { - assert(tableDefinition.catalogStats.isDefined) - // first we need to clear the old (i.e. inaccurate) stats - var propertiesWithStats: Map[String, String] = - tableDefinition.properties - (STATISTICS_TOTAL_SIZE, STATISTICS_NUM_ROWS) - val stats = tableDefinition.catalogStats.get - // convert table statistics to properties so that we can persist them through hive api - propertiesWithStats += (STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) - if (stats.rowCount.isDefined) { - propertiesWithStats += (STATISTICS_NUM_ROWS -> stats.rowCount.get.toString()) + client.alterTable(catalogTable) } - alterTable(tableDefinition.copy(properties = propertiesWithStats)) } override def getTable(db: String, table: String): CatalogTable = withClient { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index c69db59da595..c7565807d211 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -109,47 +109,39 @@ private[hive] case class MetastoreRelation( } @transient override lazy val statistics: Statistics = { - val sizeInBytes = { - val totalSize = hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE) - val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) - val analyzedSize = hiveQlTable.getParameters.get(HiveExternalCatalog.STATISTICS_TOTAL_SIZE) - // 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. - // Besides `totalSize`, there are also `numFiles`, `numRows`, `rawDataSize` keys - // (see StatsSetupConst in Hive) that we can look at in the future. - BigInt( - // 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, use `HiveExternalCatalog.STATISTICS_TOTAL_SIZE`, - // which is generated by analyze command. - // if the size is still less than zero, we try to get the file size from HDFS. - // given this is only needed for optimization, if the HDFS call fails we return the default. - if (totalSize != null && totalSize.toLong > 0L) { - totalSize.toLong - } else if (rawDataSize != null && rawDataSize.toLong > 0) { - rawDataSize.toLong - } else if (analyzedSize != null && analyzedSize.toLong > 0) { - analyzedSize.toLong - } else if (sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled) { - try { - val hadoopConf = sparkSession.sessionState.newHadoopConf() - val fs: FileSystem = hiveQlTable.getPath.getFileSystem(hadoopConf) - fs.getContentSummary(hiveQlTable.getPath).getLength - } catch { - case e: IOException => - logWarning("Failed to get table size from hdfs.", e) - sparkSession.sessionState.conf.defaultSizeInBytes - } - } else { - sparkSession.sessionState.conf.defaultSizeInBytes - }) - } - if (catalogTable.catalogStats.isDefined) { - catalogTable.catalogStats.get.copy(sizeInBytes = sizeInBytes) - } else { - Statistics(sizeInBytes) - } + catalogTable.catalogStats.getOrElse(Statistics( + sizeInBytes = { + val totalSize = hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE) + val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) + // 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. + // Besides `totalSize`, there are also `numFiles`, `numRows`, `rawDataSize` keys + // (see StatsSetupConst in Hive) that we can look at in the future. + BigInt( + // 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, use `HiveExternalCatalog.STATISTICS_TOTAL_SIZE`, + // which is generated by analyze command. + if (totalSize != null && totalSize.toLong > 0L) { + totalSize.toLong + } else if (rawDataSize != null && rawDataSize.toLong > 0) { + rawDataSize.toLong + } else if (sparkSession.sessionState.conf.fallBackToHdfsForStatsEnabled) { + try { + val hadoopConf = sparkSession.sessionState.newHadoopConf() + val fs: FileSystem = hiveQlTable.getPath.getFileSystem(hadoopConf) + fs.getContentSummary(hiveQlTable.getPath).getLength + } catch { + case e: IOException => + logWarning("Failed to get table size from hdfs.", e) + sparkSession.sessionState.conf.defaultSizeInBytes + } + } else { + sparkSession.sessionState.conf.defaultSizeInBytes + }) + } + )) } // When metastore partition pruning is turned off, we cache the list of all partitions to diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 438f9be56ddb..12a59cf8f858 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.command.AnalyzeTableCommand +import org.apache.spark.sql.execution.command.{DDLUtils, AnalyzeTableCommand} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -171,48 +171,71 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils private def checkMetastoreRelationStats( tableName: String, - totalSize: Long, - rowCount: Option[BigInt]): Unit = { + expectedTotalSize: Long, + expectedRowCount: Option[BigInt]): Unit = { val df = sql(s"SELECT * FROM $tableName") val relations = df.queryExecution.analyzed.collect { case rel: MetastoreRelation => - rel.statistics - assert(rel.statistics.sizeInBytes === totalSize) - assert(rel.statistics.rowCount === rowCount) - } - assert(relations.size === 1) - } - - private def checkLogicalRelationStats(tableName: String, rowCount: Option[BigInt]): Unit = { - val df = sql(s"SELECT * FROM $tableName") - val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation => - assert(rel.statistics.sizeInBytes === rel.relation.sizeInBytes) - assert(rel.statistics.rowCount === rowCount) + assert(rel.statistics.sizeInBytes === expectedTotalSize) + assert(rel.statistics.rowCount === expectedRowCount) + rel } assert(relations.size === 1) } test("test table-level statistics for hive tables created in HiveExternalCatalog") { val textTable = "textTable" - val parquetTable = "parquetTable" - val orcTable = "orcTable" - withTable(textTable, parquetTable, orcTable) { + withTable(textTable) { sql(s"CREATE TABLE $textTable (key STRING, value STRING) STORED AS TEXTFILE") + checkMetastoreRelationStats(textTable, + expectedTotalSize = spark.sessionState.conf.defaultSizeInBytes, expectedRowCount = None) + sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") + // don't have our statistics, MetastoreRelation uses hive's `totalSize` + checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = None) // noscan won't count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") - checkMetastoreRelationStats(textTable, 5812, None) + checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = None) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") - checkMetastoreRelationStats(textTable, 5812, Some(500)) + checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = Some(500)) + } + } + + test("test whether the old stats are removed") { + val textTable = "textTable" + withTable(textTable) { + sql(s"CREATE TABLE $textTable (key STRING, value STRING) STORED AS TEXTFILE") + sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") + checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = Some(500)) - // test whether the old stats are removed sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") - checkMetastoreRelationStats(textTable, 11624, None) + // update total size and remove the old and invalid row count + checkMetastoreRelationStats(textTable, expectedTotalSize = 11624, expectedRowCount = None) + } + } - // test statistics of LogicalRelation inherited from MetastoreRelation + private def checkLogicalRelationStats( + tableName: String, + expectedRowCount: Option[BigInt]): Unit = { + val df = sql(s"SELECT * FROM $tableName") + val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation => + // TODO: We don't have an expected value here because parquet size is different on Windows + // and Linux, we need to find the reason and fix this. + assert(rel.statistics.sizeInBytes === rel.relation.sizeInBytes) + assert(rel.statistics.rowCount === expectedRowCount) + rel + } + assert(relations.size === 1) + } + + test("test statistics of LogicalRelation inherited from MetastoreRelation") { + val parquetTable = "parquetTable" + val orcTable = "orcTable" + withTable(parquetTable, orcTable) { sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) STORED AS PARQUET") sql(s"CREATE TABLE $orcTable (key STRING, value STRING) STORED AS ORC") sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") @@ -220,10 +243,10 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") - checkLogicalRelationStats(parquetTable, Some(500)) + checkLogicalRelationStats(parquetTable, expectedRowCount = Some(500)) withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { - checkLogicalRelationStats(orcTable, Some(500)) + checkLogicalRelationStats(orcTable, expectedRowCount = Some(500)) } } } @@ -232,15 +255,18 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils val parquetTable = "parquetTable" withTable(parquetTable) { sql(s"CREATE TABLE $parquetTable (key STRING, value STRING) USING PARQUET") + val catalogTable = spark.sessionState.catalog.getTableMetadata(TableIdentifier(parquetTable)) + assert(DDLUtils.isDatasourceTable(catalogTable)) + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") // noscan won't count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") - checkLogicalRelationStats(parquetTable, None) + checkLogicalRelationStats(parquetTable, expectedRowCount = None) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") - checkLogicalRelationStats(parquetTable, Some(500)) + checkLogicalRelationStats(parquetTable, expectedRowCount = Some(500)) } } From 56ec68e455cefcc589db0336a580754662d8257c Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Wed, 31 Aug 2016 17:05:13 +0800 Subject: [PATCH 20/25] fix import order --- .../test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 12a59cf8f858..6b759b4df7cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.execution.command.{DDLUtils, AnalyzeTableCommand} +import org.apache.spark.sql.execution.command.{AnalyzeTableCommand, DDLUtils} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton From 971577086ff4a61824034f7f04b4eba0f6de7e95 Mon Sep 17 00:00:00 2001 From: Zhenhua Wang Date: Thu, 1 Sep 2016 00:44:55 +0800 Subject: [PATCH 21/25] refactor LogicalRelation --- .../sql/catalyst/catalog/interface.scala | 2 +- .../apache/spark/sql/DataFrameReader.scala | 2 +- .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../spark/sql/catalyst/SQLBuilder.scala | 8 +++-- .../execution/OptimizeMetadataOnlyQuery.scala | 4 +-- .../command/AnalyzeTableCommand.scala | 12 +++---- .../command/createDataSourceTables.scala | 2 +- .../datasources/DataSourceStrategy.scala | 25 +++++++------- .../datasources/FileSourceStrategy.scala | 4 +-- .../datasources/LogicalRelation.scala | 13 ++++--- .../sql/execution/datasources/rules.scala | 20 +++++------ .../datasources/FileSourceStrategySuite.scala | 2 +- .../parquet/ParquetFilterSuite.scala | 2 +- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../spark/sql/sources/FilteredScanSuite.scala | 2 +- .../sql/streaming/FileStreamSinkSuite.scala | 2 +- .../spark/sql/hive/HiveExternalCatalog.scala | 34 ++++++++++++++----- .../spark/sql/hive/HiveMetastoreCatalog.scala | 17 +++------- .../spark/sql/hive/MetastoreRelation.scala | 2 +- .../sql/hive/client/HiveClientImpl.scala | 17 ---------- .../sql/hive/MetastoreDataSourcesSuite.scala | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 9 +++-- .../sql/hive/execution/SQLQuerySuite.scala | 2 +- .../spark/sql/hive/orc/OrcFilterSuite.scala | 4 +-- .../apache/spark/sql/hive/parquetSuites.scala | 8 ++--- .../sql/sources/HadoopFsRelationTest.scala | 2 +- 26 files changed, 99 insertions(+), 102 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 335de8cfbd7b..aa168c8402a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -130,7 +130,7 @@ case class CatalogTable( createTime: Long = System.currentTimeMillis, lastAccessTime: Long = -1, properties: Map[String, String] = Map.empty, - catalogStats: Option[Statistics] = None, + stats: Option[Statistics] = None, viewOriginalText: Option[String] = None, viewText: Option[String] = None, comment: Option[String] = None, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index c060091c7fc3..93bf74d06b71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -26,7 +26,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.LogicalRDD -import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} +import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.execution.datasources.json.{InferSchema, JacksonParser, JSONOptions} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index b9f9f4b17c6e..6da99ce0dd68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2531,7 +2531,7 @@ class Dataset[T] private[sql]( */ def inputFiles: Array[String] = { val files: Seq[String] = logicalPlan.collect { - case LogicalRelation(fsBasedRelation: FileRelation, _, _, _) => + case LogicalRelation(fsBasedRelation: FileRelation, _, _) => fsBasedRelation.inputFiles case fr: FileRelation => fr.inputFiles diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index de8ca4e9fa49..d448d36c19d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -590,8 +590,12 @@ class SQLBuilder private ( object ExtractSQLTable { def unapply(plan: LogicalPlan): Option[SQLTable] = plan match { - case l @ LogicalRelation(_, _, Some(TableIdentifier(table, Some(database))), _) => - Some(SQLTable(database, table, l.output.map(_.withQualifier(None)))) + case l @ LogicalRelation(_, _, Some(catalogTable)) + if catalogTable.identifier.database.isDefined => + Some(SQLTable( + catalogTable.identifier.database.get, + catalogTable.identifier.table, + l.output.map(_.withQualifier(None)))) case relation: CatalogRelation => val m = relation.catalogTable diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index f244d23773a8..1b7fedca8484 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -95,7 +95,7 @@ case class OptimizeMetadataOnlyQuery( child transform { case plan if plan eq relation => relation match { - case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) => + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) val partitionData = fsRelation.location.listFiles(filters = Nil) LocalRelation(partAttrs, partitionData.map(_.values)) @@ -127,7 +127,7 @@ case class OptimizeMetadataOnlyQuery( object PartitionedRelation { def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { - case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _, _) + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) if fsRelation.partitionSchema.nonEmpty => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) Some(AttributeSet(partAttrs), l) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index c3bdd52f7a9c..ab9e82395a1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -87,16 +87,14 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend updateTableStats( catalogTable, - oldTotalSize = catalogTable.catalogStats.map(_.sizeInBytes.toLong).getOrElse(0L), - oldRowCount = catalogTable.catalogStats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L), + oldTotalSize = catalogTable.stats.map(_.sizeInBytes.toLong).getOrElse(0L), + oldRowCount = catalogTable.stats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L), newTotalSize = newTotalSize) // data source tables have been converted into LogicalRelations - case logicalRel: LogicalRelation if logicalRel.metastoreTableIdentifier.isDefined => - val tableIdentifier = logicalRel.metastoreTableIdentifier.get - val catalogTable = sessionState.catalog.getTableMetadata(tableIdentifier) + case logicalRel: LogicalRelation if logicalRel.catalogTable.isDefined => updateTableStats( - catalogTable, + logicalRel.catalogTable.get, oldTotalSize = logicalRel.statistics.sizeInBytes.toLong, oldRowCount = logicalRel.statistics.rowCount.map(_.toLong).getOrElse(-1L), newTotalSize = logicalRel.relation.sizeInBytes) @@ -128,7 +126,7 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend // Update the metastore if the above statistics of the table are different from those // recorded in the metastore. if (newStats.isDefined) { - sessionState.catalog.alterTable(catalogTable.copy(catalogStats = newStats)) + sessionState.catalog.alterTable(catalogTable.copy(stats = newStats)) // Refresh the cached data source table in the catalog. sessionState.catalog.refreshTable(tableIdent) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index 4be8a10a8a43..7400a0e7bb1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -174,7 +174,7 @@ case class CreateDataSourceTableAsSelectCommand( EliminateSubqueryAliases( sessionState.catalog.lookupRelation(tableIdent)) match { - case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _, _) => + case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => // check if the file formats match l.relation match { case r: HadoopFsRelation if r.fileFormat.getClass != dataSource.providingClass => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 81fdac117e09..9ba2148bbfe7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -144,7 +144,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { // dynamic_partitioning_columns are partitioning columns that do not assigned // values in the PARTITION clause (e.g. c in the above example). case insert @ logical.InsertIntoTable( - relation @ LogicalRelation(t: HadoopFsRelation, _, _, _), parts, query, overwrite, false) + relation @ LogicalRelation(t: HadoopFsRelation, _, _), parts, query, overwrite, false) if query.resolved && parts.exists(_._2.isDefined) => val projectList = convertStaticPartitions( @@ -159,7 +159,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: HadoopFsRelation, _, _, _), part, query, overwrite, false) + l @ LogicalRelation(t: HadoopFsRelation, _, _), part, query, overwrite, false) if query.resolved && t.schema.asNullable == query.schema.asNullable => // Sanity checks @@ -170,7 +170,7 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { val outputPath = t.location.paths.head val inputPaths = query.collect { - case LogicalRelation(r: HadoopFsRelation, _, _, _) => r.location.paths + case LogicalRelation(r: HadoopFsRelation, _, _) => r.location.paths }.flatten val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append @@ -209,8 +209,7 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] LogicalRelation( dataSource.resolveRelation(), - metastoreTableIdentifier = Some(table.identifier), - inheritedStats = table.catalogStats) + catalogTable = Some(table)) } override def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -229,7 +228,7 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] */ object DataSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[execution.SparkPlan] = plan match { - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: CatalystScan, _, _)) => pruneFilterProjectRaw( l, projects, @@ -238,21 +237,21 @@ object DataSourceStrategy extends Strategy with Logging { toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, allPredicates))) :: Nil case PhysicalOperation(projects, filters, - l @ LogicalRelation(t: PrunedFilteredScan, _, _, _)) => + l @ LogicalRelation(t: PrunedFilteredScan, _, _)) => pruneFilterProject( l, projects, filters, (a, f) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f))) :: Nil - case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedScan, _, _)) => pruneFilterProject( l, projects, filters, (a, _) => toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray))) :: Nil - case l @ LogicalRelation(baseRelation: TableScan, _, _, _) => + case l @ LogicalRelation(baseRelation: TableScan, _, _) => RowDataSourceScanExec( l.output, toCatalystRDD(l, baseRelation.buildScan()), @@ -261,7 +260,7 @@ object DataSourceStrategy extends Strategy with Logging { Map.empty, None) :: Nil - case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _, _), + case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), part, query, overwrite, false) if part.isEmpty => ExecutedCommandExec(InsertIntoDataSourceCommand(l, query, overwrite)) :: Nil @@ -368,7 +367,8 @@ object DataSourceStrategy extends Strategy with Logging { val scan = RowDataSourceScanExec( projects.map(_.toAttribute), scanBuilder(requestedColumns, candidatePredicates, pushedFilters), - relation.relation, UnknownPartitioning(0), metadata, relation.metastoreTableIdentifier) + relation.relation, UnknownPartitioning(0), metadata, + relation.catalogTable.map(_.identifier)) filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan) } else { // Don't request columns that are only referenced by pushed filters. @@ -378,7 +378,8 @@ object DataSourceStrategy extends Strategy with Logging { val scan = RowDataSourceScanExec( requestedColumns, scanBuilder(requestedColumns, candidatePredicates, pushedFilters), - relation.relation, UnknownPartitioning(0), metadata, relation.metastoreTableIdentifier) + relation.relation, UnknownPartitioning(0), metadata, + relation.catalogTable.map(_.identifier)) execution.ProjectExec( projects, filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 7ad7a249e7d4..55ca4f11068f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -52,7 +52,7 @@ import org.apache.spark.sql.execution.SparkPlan object FileSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projects, filters, - l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table, _)) => + l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table)) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: // - partition keys only - used to prune directories to read @@ -112,7 +112,7 @@ object FileSourceStrategy extends Strategy with Logging { outputSchema, partitionKeyFilters.toSeq, pushedDownFilters, - table) + table.map(_.identifier)) val afterScanFilter = afterScanFilters.toSeq.reduceOption(expressions.And) val withFilter = afterScanFilter.map(execution.FilterExec(_, scan)).getOrElse(scan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 3fb2eb486af8..f7baed21e781 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -16,8 +16,8 @@ */ package org.apache.spark.sql.execution.datasources -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.sources.BaseRelation @@ -33,8 +33,7 @@ import org.apache.spark.util.Utils case class LogicalRelation( relation: BaseRelation, expectedOutputAttributes: Option[Seq[Attribute]] = None, - metastoreTableIdentifier: Option[TableIdentifier] = None, - inheritedStats: Option[Statistics] = None) + catalogTable: Option[CatalogTable] = None) extends LeafNode with MultiInstanceRelation { override val output: Seq[AttributeReference] = { @@ -53,7 +52,7 @@ case class LogicalRelation( // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any): Boolean = other match { - case l @ LogicalRelation(otherRelation, _, _, _) => + case l @ LogicalRelation(otherRelation, _, _) => relation == otherRelation && output == l.output case _ => false } @@ -64,7 +63,7 @@ case class LogicalRelation( override def sameResult(otherPlan: LogicalPlan): Boolean = { otherPlan.canonicalized match { - case LogicalRelation(otherRelation, _, _, _) => relation == otherRelation + case LogicalRelation(otherRelation, _, _) => relation == otherRelation case _ => false } } @@ -76,7 +75,7 @@ case class LogicalRelation( // inheritedStats is inherited from a CatalogRelation @transient override lazy val statistics: Statistics = { - inheritedStats.map(_.copy(sizeInBytes = relation.sizeInBytes)).getOrElse( + catalogTable.flatMap(_.stats.map(_.copy(sizeInBytes = relation.sizeInBytes))).getOrElse( Statistics(sizeInBytes = relation.sizeInBytes)) } @@ -93,7 +92,7 @@ case class LogicalRelation( LogicalRelation( relation, expectedOutputAttributes.map(_.map(_.newInstance())), - metastoreTableIdentifier).asInstanceOf[this.type] + catalogTable).asInstanceOf[this.type] } override def refresh(): Unit = relation match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 829f69983684..249c083856f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -252,11 +252,11 @@ case class PreprocessTableInsertion(conf: SQLConf) extends Rule[LogicalPlan] { case relation: CatalogRelation => val metadata = relation.catalogTable preprocess(i, metadata.identifier.quotedString, metadata.partitionColumnNames) - case LogicalRelation(h: HadoopFsRelation, _, identifier, _) => - val tblName = identifier.map(_.quotedString).getOrElse("unknown") + case LogicalRelation(h: HadoopFsRelation, _, catalogTable) => + val tblName = catalogTable.map(_.identifier.quotedString).getOrElse("unknown") preprocess(i, tblName, h.partitionSchema.map(_.name)) - case LogicalRelation(_: InsertableRelation, _, identifier, _) => - val tblName = identifier.map(_.quotedString).getOrElse("unknown") + case LogicalRelation(_: InsertableRelation, _, catalogTable) => + val tblName = catalogTable.map(_.identifier.quotedString).getOrElse("unknown") preprocess(i, tblName, Nil) case other => i } @@ -306,7 +306,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation, _, _, _), + l @ LogicalRelation(t: InsertableRelation, _, _), partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { @@ -314,7 +314,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } else { // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _, _, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(t)) { failAnalysis( @@ -325,7 +325,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) } case logical.InsertIntoTable( - LogicalRelation(r: HadoopFsRelation, _, _, _), part, query, overwrite, _) => + LogicalRelation(r: HadoopFsRelation, _, _), part, query, overwrite, _) => // We need to make sure the partition columns specified by users do match partition // columns of the relation. val existingPartitionColumns = r.partitionSchema.fieldNames.toSet @@ -344,7 +344,7 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _, _, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(r)) { failAnalysis( @@ -365,10 +365,10 @@ case class PreWriteCheck(conf: SQLConf, catalog: SessionCatalog) EliminateSubqueryAliases(catalog.lookupRelation(tableDesc.identifier)) match { // Only do the check if the table is a data source table // (the relation is a BaseRelation). - case l @ LogicalRelation(dest: BaseRelation, _, _, _) => + case l @ LogicalRelation(dest: BaseRelation, _, _) => // Get all input data source relations of the query. val srcRelations = query.collect { - case LogicalRelation(src: BaseRelation, _, _, _) => src + case LogicalRelation(src: BaseRelation, _, _) => src } if (srcRelations.contains(dest)) { failAnalysis( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 2877754993a2..09fd75018035 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -506,7 +506,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi if (buckets > 0) { val bucketed = df.queryExecution.analyzed transform { - case l @ LogicalRelation(r: HadoopFsRelation, _, _, _) => + case l @ LogicalRelation(r: HadoopFsRelation, _, _) => l.copy(relation = r.copy(bucketSpec = Some(BucketSpec(numBuckets = buckets, "c1" :: Nil, Nil)))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index ac1165c8adfa..b697298f9a52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -64,7 +64,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(relation: HadoopFsRelation, _, _, _) + case PhysicalOperation(_, filters, LogicalRelation(relation: HadoopFsRelation, _, _) ) => maybeRelation = Some(relation) filters diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 5e4b420d0da5..8d18be9300f7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -626,7 +626,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { - case LogicalRelation(relation: HadoopFsRelation, _, _, _) => + case LogicalRelation(relation: HadoopFsRelation, _, _) => assert(relation.partitionSpec === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a ParquetRelation2, but got:\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 461e4fb85c55..be56c964a18f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -324,7 +324,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic val table = spark.table("oneToTenFiltered") val relation = table.queryExecution.logical.collectFirst { - case LogicalRelation(r, _, _, _) => r + case LogicalRelation(r, _, _) => r }.get assert( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index b36656853da0..8b73fd697f77 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -182,7 +182,7 @@ class FileStreamSinkSuite extends StreamTest { // Verify that MetadataLogFileCatalog is being used and the correct partitioning schema has // been inferred val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { - case LogicalRelation(baseRelation, _, _, _) + case LogicalRelation(baseRelation, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => baseRelation.asInstanceOf[HadoopFsRelation] } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index bc609759ff1a..ed92a131e269 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.CaseInsensitiveMap import org.apache.spark.sql.hive.client.HiveClient @@ -384,16 +385,14 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat verifyTableProperties(tableDefinition) // convert table statistics to properties so that we can persist them through hive api - val catalogTable = if (tableDefinition.catalogStats.isDefined) { - // first we need to clear the old (i.e. inaccurate) stats - var propertiesWithStats: Map[String, String] = - tableDefinition.properties -(STATISTICS_TOTAL_SIZE, STATISTICS_NUM_ROWS) - val stats = tableDefinition.catalogStats.get - propertiesWithStats += (STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) + val catalogTable = 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) { - propertiesWithStats += (STATISTICS_NUM_ROWS -> stats.rowCount.get.toString()) + statsProperties += (STATISTICS_NUM_ROWS -> stats.rowCount.get.toString()) } - tableDefinition.copy(properties = propertiesWithStats) + tableDefinition.copy(properties = tableDefinition.properties ++ statsProperties) } else { tableDefinition } @@ -432,7 +431,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat * properties, and filter out these special entries from table properties. */ private def restoreTableMetadata(table: CatalogTable): CatalogTable = { - if (table.tableType == VIEW) { + val catalogTable = if (table.tableType == VIEW) { table } else { getProviderFromTableProperties(table).map { provider => @@ -462,6 +461,19 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat table.copy(provider = Some("hive")) } } + // construct Spark's statistics from information in Hive metastore + if (catalogTable.properties.contains(STATISTICS_TOTAL_SIZE)) { + val totalSize = BigInt(catalogTable.properties.get(STATISTICS_TOTAL_SIZE).get) + // TODO: we will compute "estimatedSize" when we have column stats: + // average size of row * number of rows + catalogTable.copy( + properties = removeStatsProperties(catalogTable), + stats = Some(Statistics( + sizeInBytes = totalSize, + rowCount = catalogTable.properties.get(STATISTICS_NUM_ROWS).map(BigInt(_))))) + } else { + catalogTable + } } override def tableExists(db: String, table: String): Boolean = withClient { @@ -629,6 +641,10 @@ object HiveExternalCatalog { val STATISTICS_TOTAL_SIZE = STATISTICS_PREFIX + "totalSize" val STATISTICS_NUM_ROWS = STATISTICS_PREFIX + "numRows" + def removeStatsProperties(metadata: CatalogTable): Map[String, String] = { + metadata.properties.filterNot { case (key, _) => key.startsWith(STATISTICS_PREFIX) } + } + def getProviderFromTableProperties(metadata: CatalogTable): Option[String] = { metadata.properties.get(DATASOURCE_PROVIDER) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 58693ae3d7fd..d31a8d643ad8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -82,8 +82,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log LogicalRelation( dataSource.resolveRelation(checkPathExist = true), - metastoreTableIdentifier = Some(TableIdentifier(in.name, Some(in.database))), - inheritedStats = table.catalogStats) + catalogTable = Some(table)) } } @@ -146,7 +145,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log cachedDataSourceTables.getIfPresent(tableIdentifier) match { case null => None // Cache miss - case logical @ LogicalRelation(relation: HadoopFsRelation, _, _, _) => + case logical @ LogicalRelation(relation: HadoopFsRelation, _, _) => val cachedRelationFileFormatClass = relation.fileFormat.getClass expectedFileFormat match { @@ -258,10 +257,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log fileFormat = defaultSource, options = options) - val created = LogicalRelation( - relation, - metastoreTableIdentifier = - Some(TableIdentifier(tableIdentifier.name, Some(tableIdentifier.database)))) + val created = LogicalRelation(relation, catalogTable = Some(metastoreRelation.catalogTable)) cachedDataSourceTables.put(tableIdentifier, created) created } @@ -287,8 +283,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log bucketSpec = bucketSpec, options = options, className = fileType).resolveRelation(), - metastoreTableIdentifier = - Some(TableIdentifier(tableIdentifier.name, Some(tableIdentifier.database)))) + catalogTable = Some(metastoreRelation.catalogTable)) cachedDataSourceTables.put(tableIdentifier, created) @@ -297,9 +292,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log logicalRelation } - result.copy( - expectedOutputAttributes = Some(metastoreRelation.output), - inheritedStats = Some(metastoreRelation.statistics)) + result.copy(expectedOutputAttributes = Some(metastoreRelation.output)) } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala index c7565807d211..ca38a0a03101 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala @@ -109,7 +109,7 @@ private[hive] case class MetastoreRelation( } @transient override lazy val statistics: Statistics = { - catalogTable.catalogStats.getOrElse(Statistics( + catalogTable.stats.getOrElse(Statistics( sizeInBytes = { val totalSize = hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE) val rawDataSize = hiveQlTable.getParameters.get(StatsSetupConst.RAW_DATA_SIZE) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index d9258b388003..b45ad30dcae4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -44,9 +44,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} -import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.{CircularBuffer, Utils} @@ -403,7 +401,6 @@ private[hive] class HiveClientImpl( .map(_.asScala.toMap).orNull ), properties = properties.filter(kv => kv._1 != "comment"), - catalogStats = constructStatsFromHive(properties), comment = properties.get("comment"), viewOriginalText = Option(h.getViewOriginalText), viewText = Option(h.getViewExpandedText), @@ -430,20 +427,6 @@ private[hive] class HiveClientImpl( client.alterTable(qualifiedTableName, hiveTable) } - // Construct Spark's statistics from information in Hive metastore. - private def constructStatsFromHive(properties: Map[String, String]): Option[Statistics] = { - if (properties.contains(HiveExternalCatalog.STATISTICS_TOTAL_SIZE)) { - val totalSize = BigInt(properties.get(HiveExternalCatalog.STATISTICS_TOTAL_SIZE).get) - // TODO: we will compute "estimatedSize" when we have column stats: - // average size of row * number of rows - Some(Statistics( - sizeInBytes = totalSize, - rowCount = properties.get(HiveExternalCatalog.STATISTICS_NUM_ROWS).map(BigInt(_)))) - } else { - None - } - } - override def createPartitions( db: String, table: String, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index acaed4ec191c..d77bb5cf95f6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -572,7 +572,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv Row(3) :: Row(4) :: Nil) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(p: HadoopFsRelation, _, _, _) => // OK + case LogicalRelation(p: HadoopFsRelation, _, _) => // OK case _ => fail(s"test_parquet_ctas should have be converted to ${classOf[HadoopFsRelation]}") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 6b759b4df7cb..11de7524b45b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -203,7 +203,7 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils } } - test("test whether the old stats are removed") { + test("test elimination of the influences of the old stats") { val textTable = "textTable" withTable(textTable) { sql(s"CREATE TABLE $textTable (key STRING, value STRING) STORED AS TEXTFILE") @@ -243,8 +243,11 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") - checkLogicalRelationStats(parquetTable, expectedRowCount = Some(500)) - + // the default value for `spark.sql.hive.convertMetastoreParquet` is true, here we just set it + // for robustness + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "true") { + checkLogicalRelationStats(parquetTable, expectedRowCount = Some(500)) + } withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { checkLogicalRelationStats(orcTable, expectedRowCount = Some(500)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 839028ad8543..4ca882f840a5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -424,7 +424,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { val catalogTable = sessionState.catalog.getTableMetadata(TableIdentifier(tableName)) relation match { - case LogicalRelation(r: HadoopFsRelation, _, _, _) => + case LogicalRelation(r: HadoopFsRelation, _, _) => if (!isDataSourceParquet) { fail( s"${classOf[MetastoreRelation].getCanonicalName} is expected, but found " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala index 89ffe1442ddc..471192a369f4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala @@ -45,7 +45,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) @@ -89,7 +89,7 @@ class OrcFilterSuite extends QueryTest with OrcTest { var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _)) => maybeRelation = Some(orcRelation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 1d11ccb7faf9..e92bbdea75a7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -284,7 +284,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { ) table("test_parquet_ctas").queryExecution.optimizedPlan match { - case LogicalRelation(_: HadoopFsRelation, _, _, _) => // OK + case LogicalRelation(_: HadoopFsRelation, _, _) => // OK case _ => fail( "test_parquet_ctas should be converted to " + s"${classOf[HadoopFsRelation ].getCanonicalName }") @@ -371,7 +371,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { assertResult(2) { analyzed.collect { - case r @ LogicalRelation(_: HadoopFsRelation, _, _, _) => r + case r @ LogicalRelation(_: HadoopFsRelation, _, _) => r }.size } } @@ -380,7 +380,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { def collectHadoopFsRelation(df: DataFrame): HadoopFsRelation = { val plan = df.queryExecution.analyzed plan.collectFirst { - case LogicalRelation(r: HadoopFsRelation, _, _, _) => r + case LogicalRelation(r: HadoopFsRelation, _, _) => r }.getOrElse { fail(s"Expecting a HadoopFsRelation 2, but got:\n$plan") } @@ -455,7 +455,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { // Converted test_parquet should be cached. sessionState.catalog.getCachedDataSourceTable(tableIdentifier) match { case null => fail("Converted test_parquet should be cached in the cache.") - case logical @ LogicalRelation(parquetRelation: HadoopFsRelation, _, _, _) => // OK + case logical @ LogicalRelation(parquetRelation: HadoopFsRelation, _, _) => // OK case other => fail( "The cached test_parquet should be a Parquet Relation. " + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 27bb9676e9ab..63d0b97945eb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.execution.DataSourceScanExec -import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{FileScanRDD, LocalityTestFileSystem} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils From aa438c43f78d5edd679fd3e6294d953181a40268 Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Thu, 1 Sep 2016 10:02:22 +0800 Subject: [PATCH 22/25] remove unnecessary changes --- .../src/main/scala/org/apache/spark/sql/DataFrameReader.scala | 2 +- .../sql/execution/datasources/parquet/ParquetFilterSuite.scala | 3 +-- .../org/apache/spark/sql/streaming/FileStreamSinkSuite.scala | 3 +-- .../org/apache/spark/sql/sources/HadoopFsRelationTest.scala | 2 +- 4 files changed, 4 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 93bf74d06b71..c060091c7fc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -26,7 +26,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.LogicalRDD -import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.execution.datasources.json.{InferSchema, JacksonParser, JSONOptions} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index b697298f9a52..4246b54c21f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -64,8 +64,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex var maybeRelation: Option[HadoopFsRelation] = None val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(relation: HadoopFsRelation, _, _) - ) => + case PhysicalOperation(_, filters, LogicalRelation(relation: HadoopFsRelation, _, _)) => maybeRelation = Some(relation) filters }.flatten.reduceLeftOption(_ && _) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 8b73fd697f77..19c89f5c4100 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -182,8 +182,7 @@ class FileStreamSinkSuite extends StreamTest { // Verify that MetadataLogFileCatalog is being used and the correct partitioning schema has // been inferred val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { - case LogicalRelation(baseRelation, _, _) - if baseRelation.isInstanceOf[HadoopFsRelation] => + case LogicalRelation(baseRelation, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => baseRelation.asInstanceOf[HadoopFsRelation] } assert(hadoopdFsRelations.size === 1) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 63d0b97945eb..27bb9676e9ab 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -29,7 +29,7 @@ import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql._ import org.apache.spark.sql.execution.DataSourceScanExec -import org.apache.spark.sql.execution.datasources.{FileScanRDD, LocalityTestFileSystem} +import org.apache.spark.sql.execution.datasources.{FileScanRDD, HadoopFsRelation, LocalityTestFileSystem, LogicalRelation} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils From b6c655a176c313aa8aa055ef985401f78557e4ec Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Thu, 1 Sep 2016 22:51:37 +0800 Subject: [PATCH 23/25] fix bug about size --- .../spark/sql/catalyst/SQLBuilder.scala | 2 +- .../command/AnalyzeTableCommand.scala | 7 +- .../datasources/LogicalRelation.scala | 4 +- .../apache/spark/sql/StatisticsSuite.scala | 10 +-- .../spark/sql/hive/StatisticsSuite.scala | 78 +++++++++++++------ 5 files changed, 64 insertions(+), 37 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala index d448d36c19d1..6f821f80cc4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/SQLBuilder.scala @@ -591,7 +591,7 @@ class SQLBuilder private ( object ExtractSQLTable { def unapply(plan: LogicalPlan): Option[SQLTable] = plan match { case l @ LogicalRelation(_, _, Some(catalogTable)) - if catalogTable.identifier.database.isDefined => + if catalogTable.identifier.database.isDefined => Some(SQLTable( catalogTable.identifier.database.get, catalogTable.identifier.table, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index ab9e82395a1e..cc003f143593 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -93,10 +93,11 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend // data source tables have been converted into LogicalRelations case logicalRel: LogicalRelation if logicalRel.catalogTable.isDefined => + val table = logicalRel.catalogTable.get updateTableStats( - logicalRel.catalogTable.get, - oldTotalSize = logicalRel.statistics.sizeInBytes.toLong, - oldRowCount = logicalRel.statistics.rowCount.map(_.toLong).getOrElse(-1L), + table, + oldTotalSize = table.stats.map(_.sizeInBytes.toLong).getOrElse(0L), + oldRowCount = table.stats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L), newTotalSize = logicalRel.relation.sizeInBytes) case otherRelation => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index f7baed21e781..d9562fd32e87 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -52,8 +52,7 @@ case class LogicalRelation( // Logical Relations are distinct if they have different output for the sake of transformations. override def equals(other: Any): Boolean = other match { - case l @ LogicalRelation(otherRelation, _, _) => - relation == otherRelation && output == l.output + case l @ LogicalRelation(otherRelation, _, _) => relation == otherRelation && output == l.output case _ => false } @@ -73,7 +72,6 @@ case class LogicalRelation( // expId can be different but the relation is still the same. override lazy val cleanArgs: Seq[Any] = Seq(relation) - // inheritedStats is inherited from a CatalogRelation @transient override lazy val statistics: Statistics = { catalogTable.flatMap(_.stats.map(_.copy(sizeInBytes = relation.sizeInBytes))).getOrElse( Statistics(sizeInBytes = relation.sizeInBytes)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala index ee30d186c691..264a2ffbebeb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala @@ -77,11 +77,11 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { } test("test table-level statistics for data source table created in InMemoryCatalog") { - def checkTableStats(tableName: String, rowCount: Option[BigInt]): Unit = { + def checkTableStats(tableName: String, expectedRowCount: Option[BigInt]): Unit = { val df = sql(s"SELECT * FROM $tableName") val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation => - assert(rel.statistics.sizeInBytes === rel.relation.sizeInBytes) - assert(rel.statistics.rowCount === rowCount) + assert(rel.catalogTable.isDefined) + assert(rel.catalogTable.get.stats.flatMap(_.rowCount) === expectedRowCount) rel } assert(relations.size === 1) @@ -94,11 +94,11 @@ class StatisticsSuite extends QueryTest with SharedSQLContext { // noscan won't count the number of rows sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan") - checkTableStats(tableName, None) + checkTableStats(tableName, expectedRowCount = None) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS") - checkTableStats(tableName, Some(2)) + checkTableStats(tableName, expectedRowCount = Some(2)) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 11de7524b45b..5ffb9b16fe82 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.command.{AnalyzeTableCommand, DDLUtils} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.joins._ @@ -171,12 +172,18 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils private def checkMetastoreRelationStats( tableName: String, - expectedTotalSize: Long, - expectedRowCount: Option[BigInt]): Unit = { + expectedStats: Option[Statistics]): Unit = { val df = sql(s"SELECT * FROM $tableName") val relations = df.queryExecution.analyzed.collect { case rel: MetastoreRelation => - assert(rel.statistics.sizeInBytes === expectedTotalSize) - assert(rel.statistics.rowCount === expectedRowCount) + expectedStats match { + case Some(es) => + assert(rel.catalogTable.stats.isDefined) + val stats = rel.catalogTable.stats.get + assert(stats.sizeInBytes === es.sizeInBytes) + assert(stats.rowCount === es.rowCount) + case None => + assert(rel.catalogTable.stats.isEmpty) + } rel } assert(relations.size === 1) @@ -185,21 +192,22 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils test("test table-level statistics for hive tables created in HiveExternalCatalog") { val textTable = "textTable" withTable(textTable) { + // Currently Spark's statistics are self-contained, we don't have statistics until we use + // the `ANALYZE TABLE` command. sql(s"CREATE TABLE $textTable (key STRING, value STRING) STORED AS TEXTFILE") - checkMetastoreRelationStats(textTable, - expectedTotalSize = spark.sessionState.conf.defaultSizeInBytes, expectedRowCount = None) - + checkMetastoreRelationStats(textTable, expectedStats = None) sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") - // don't have our statistics, MetastoreRelation uses hive's `totalSize` - checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = None) + checkMetastoreRelationStats(textTable, expectedStats = None) // noscan won't count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") - checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = None) + checkMetastoreRelationStats(textTable, expectedStats = + Some(Statistics(sizeInBytes = 5812, rowCount = None))) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") - checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = Some(500)) + checkMetastoreRelationStats(textTable, expectedStats = + Some(Statistics(sizeInBytes = 5812, rowCount = Some(500)))) } } @@ -209,30 +217,38 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils sql(s"CREATE TABLE $textTable (key STRING, value STRING) STORED AS TEXTFILE") sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS") - checkMetastoreRelationStats(textTable, expectedTotalSize = 5812, expectedRowCount = Some(500)) + checkMetastoreRelationStats(textTable, expectedStats = + Some(Statistics(sizeInBytes = 5812, rowCount = Some(500)))) sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") // update total size and remove the old and invalid row count - checkMetastoreRelationStats(textTable, expectedTotalSize = 11624, expectedRowCount = None) + checkMetastoreRelationStats(textTable, expectedStats = + Some(Statistics(sizeInBytes = 11624, rowCount = None))) } } private def checkLogicalRelationStats( tableName: String, - expectedRowCount: Option[BigInt]): Unit = { + expectedStats: Option[Statistics]): Unit = { val df = sql(s"SELECT * FROM $tableName") val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation => - // TODO: We don't have an expected value here because parquet size is different on Windows - // and Linux, we need to find the reason and fix this. - assert(rel.statistics.sizeInBytes === rel.relation.sizeInBytes) - assert(rel.statistics.rowCount === expectedRowCount) + assert(rel.catalogTable.isDefined) + expectedStats match { + case Some(es) => + assert(rel.catalogTable.get.stats.isDefined) + val stats = rel.catalogTable.get.stats.get + assert(stats.sizeInBytes === es.sizeInBytes) + assert(stats.rowCount === es.rowCount) + case None => + assert(rel.catalogTable.get.stats.isEmpty) + } rel } assert(relations.size === 1) } - test("test statistics of LogicalRelation inherited from MetastoreRelation") { + test("test statistics of LogicalRelation converted from MetastoreRelation") { val parquetTable = "parquetTable" val orcTable = "orcTable" withTable(parquetTable, orcTable) { @@ -240,16 +256,20 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils sql(s"CREATE TABLE $orcTable (key STRING, value STRING) STORED AS ORC") sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") sql(s"INSERT INTO TABLE $orcTable SELECT * FROM src") - sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") - sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") // the default value for `spark.sql.hive.convertMetastoreParquet` is true, here we just set it // for robustness withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "true") { - checkLogicalRelationStats(parquetTable, expectedRowCount = Some(500)) + checkLogicalRelationStats(parquetTable, expectedStats = None) + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") + checkLogicalRelationStats(parquetTable, expectedStats = + Some(Statistics(sizeInBytes = 4236, rowCount = Some(500)))) } withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "true") { - checkLogicalRelationStats(orcTable, expectedRowCount = Some(500)) + checkLogicalRelationStats(orcTable, expectedStats = None) + sql(s"ANALYZE TABLE $orcTable COMPUTE STATISTICS") + checkLogicalRelationStats(orcTable, expectedStats = + Some(Statistics(sizeInBytes = 3023, rowCount = Some(500)))) } } } @@ -262,14 +282,22 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils assert(DDLUtils.isDatasourceTable(catalogTable)) sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") + checkLogicalRelationStats(parquetTable, expectedStats = None) // noscan won't count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") - checkLogicalRelationStats(parquetTable, expectedRowCount = None) + checkLogicalRelationStats(parquetTable, expectedStats = + Some(Statistics(sizeInBytes = 4236, rowCount = None))) + + sql(s"INSERT INTO TABLE $parquetTable SELECT * FROM src") + sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS noscan") + checkLogicalRelationStats(parquetTable, expectedStats = + Some(Statistics(sizeInBytes = 8472, rowCount = None))) // without noscan, we count the number of rows sql(s"ANALYZE TABLE $parquetTable COMPUTE STATISTICS") - checkLogicalRelationStats(parquetTable, expectedRowCount = Some(500)) + checkLogicalRelationStats(parquetTable, expectedStats = + Some(Statistics(sizeInBytes = 8472, rowCount = Some(1000)))) } } From b946df0928da061ca67b93d2587c1e258fd5d63b Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Fri, 2 Sep 2016 17:22:12 +0800 Subject: [PATCH 24/25] update based on comments --- .../sql/catalyst/catalog/interface.scala | 1 + .../catalyst/plans/logical/Statistics.scala | 11 +++++++- .../command/AnalyzeTableCommand.scala | 24 ++++++----------- .../datasources/DataSourceStrategy.scala | 3 +-- .../spark/sql/hive/HiveExternalCatalog.scala | 22 ++++++++------- .../sql/hive/execution/HiveDDLSuite.scala | 27 ++++++++++--------- 6 files changed, 47 insertions(+), 41 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index aa168c8402a6..79231ee9e378 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -191,6 +191,7 @@ case class CatalogTable( viewText.map("View: " + _).getOrElse(""), comment.map("Comment: " + _).getOrElse(""), if (properties.nonEmpty) s"Properties: $tableProperties" else "", + if (stats.isDefined) s"Statistics: ${stats.get}" else "", s"$storage") output.filter(_.nonEmpty).mkString("CatalogTable(\n\t", "\n\t", ")") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index d1625962ceba..58fa537a18e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -37,4 +37,13 @@ package org.apache.spark.sql.catalyst.plans.logical case class Statistics( sizeInBytes: BigInt, rowCount: Option[BigInt] = None, - isBroadcastable: Boolean = false) + isBroadcastable: Boolean = false) { + override def toString: String = { + val output = + Seq(s"sizeInBytes=$sizeInBytes", + if (rowCount.isDefined) s"rowCount=${rowCount.get}" else "", + s"isBroadcastable=$isBroadcastable" + ) + output.filter(_.nonEmpty).mkString("Statistics(", ", ", ")") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index cc003f143593..d427cd219b9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -85,35 +85,27 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend } }.getOrElse(0L) - updateTableStats( - catalogTable, - oldTotalSize = catalogTable.stats.map(_.sizeInBytes.toLong).getOrElse(0L), - oldRowCount = catalogTable.stats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L), - newTotalSize = newTotalSize) + updateTableStats(catalogTable, newTotalSize) // data source tables have been converted into LogicalRelations case logicalRel: LogicalRelation if logicalRel.catalogTable.isDefined => - val table = logicalRel.catalogTable.get - updateTableStats( - table, - oldTotalSize = table.stats.map(_.sizeInBytes.toLong).getOrElse(0L), - oldRowCount = table.stats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L), - newTotalSize = logicalRel.relation.sizeInBytes) + updateTableStats(logicalRel.catalogTable.get, logicalRel.relation.sizeInBytes) case otherRelation => throw new AnalysisException(s"ANALYZE TABLE is not supported for " + s"${otherRelation.nodeName}.") } - def updateTableStats( - catalogTable: CatalogTable, - oldTotalSize: Long, - oldRowCount: Long, - newTotalSize: Long): Unit = { + def updateTableStats(catalogTable: CatalogTable, newTotalSize: Long): Unit = { + val oldTotalSize = catalogTable.stats.map(_.sizeInBytes.toLong).getOrElse(0L) + val oldRowCount = catalogTable.stats.flatMap(_.rowCount.map(_.toLong)).getOrElse(-1L) var newStats: Option[Statistics] = None if (newTotalSize > 0 && newTotalSize != oldTotalSize) { newStats = Some(Statistics(sizeInBytes = newTotalSize)) } + // We only set rowCount when noscan is false, because otherwise we can't know whether the + // row count we get (`oldRowCount`) is valid or not. + // This is to make sure that we only record the right statistics. if (!noscan) { val newRowCount = Dataset.ofRows(sparkSession, relation).count() if (newRowCount >= 0 && newRowCount != oldRowCount) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 9ba2148bbfe7..0f7642c2afb7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -236,8 +236,7 @@ object DataSourceStrategy extends Strategy with Logging { (requestedColumns, allPredicates, _) => toCatalystRDD(l, requestedColumns, t.buildScan(requestedColumns, allPredicates))) :: Nil - case PhysicalOperation(projects, filters, - l @ LogicalRelation(t: PrunedFilteredScan, _, _)) => + case PhysicalOperation(projects, filters, l @ LogicalRelation(t: PrunedFilteredScan, _, _)) => pruneFilterProject( l, projects, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index ed92a131e269..2d080abfd836 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -102,11 +102,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat * metastore. */ private def verifyTableProperties(table: CatalogTable): Unit = { - val datasourceKeys = table.properties.keys.filter(_.startsWith(DATASOURCE_PREFIX)) - if (datasourceKeys.nonEmpty) { + val invalidKeys = table.properties.keys.filter { key => + key.startsWith(DATASOURCE_PREFIX) || key.startsWith(STATISTICS_PREFIX) + } + if (invalidKeys.nonEmpty) { throw new AnalysisException(s"Cannot persistent ${table.qualifiedName} into hive metastore " + - s"as table property keys may not start with '$DATASOURCE_PREFIX': " + - datasourceKeys.mkString("[", ", ", "]")) + s"as table property keys may not start with '$DATASOURCE_PREFIX' or '$STATISTICS_PREFIX':" + + s" ${invalidKeys.mkString("[", ", ", "]")}") } } @@ -385,7 +387,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat verifyTableProperties(tableDefinition) // convert table statistics to properties so that we can persist them through hive api - val catalogTable = if (tableDefinition.stats.isDefined) { + val withStatsProps = if (tableDefinition.stats.isDefined) { val stats = tableDefinition.stats.get var statsProperties: Map[String, String] = Map(STATISTICS_TOTAL_SIZE -> stats.sizeInBytes.toString()) @@ -397,21 +399,21 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat tableDefinition } - if (DDLUtils.isDatasourceTable(catalogTable)) { - val oldDef = client.getTable(db, catalogTable.identifier.table) + if (DDLUtils.isDatasourceTable(withStatsProps)) { + val oldDef = client.getTable(db, withStatsProps.identifier.table) // 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 = oldDef.properties.filter(_._1.startsWith(DATASOURCE_PREFIX)) - val newDef = catalogTable.copy( + val newDef = withStatsProps.copy( schema = oldDef.schema, partitionColumnNames = oldDef.partitionColumnNames, bucketSpec = oldDef.bucketSpec, - properties = oldDataSourceProps ++ catalogTable.properties) + properties = oldDataSourceProps ++ withStatsProps.properties) client.alterTable(newDef) } else { - client.alterTable(catalogTable) + client.alterTable(withStatsProps) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 9019333d7686..0d918e687a05 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -731,26 +731,29 @@ class HiveDDLSuite } } - test("datasource table property keys are not allowed") { + test("datasource and statistics table property keys are not allowed") { import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_PREFIX + import org.apache.spark.sql.hive.HiveExternalCatalog.STATISTICS_PREFIX withTable("tbl") { sql("CREATE TABLE tbl(a INT) STORED AS parquet") - val e = intercept[AnalysisException] { - sql(s"ALTER TABLE tbl SET TBLPROPERTIES ('${DATASOURCE_PREFIX}foo' = 'loser')") - } - assert(e.getMessage.contains(DATASOURCE_PREFIX + "foo")) + Seq(DATASOURCE_PREFIX, STATISTICS_PREFIX).foreach { forbiddenPrefix => + val e = intercept[AnalysisException] { + sql(s"ALTER TABLE tbl SET TBLPROPERTIES ('${forbiddenPrefix}foo' = 'loser')") + } + assert(e.getMessage.contains(forbiddenPrefix + "foo")) - val e2 = intercept[AnalysisException] { - sql(s"ALTER TABLE tbl UNSET TBLPROPERTIES ('${DATASOURCE_PREFIX}foo')") - } - assert(e2.getMessage.contains(DATASOURCE_PREFIX + "foo")) + val e2 = intercept[AnalysisException] { + sql(s"ALTER TABLE tbl UNSET TBLPROPERTIES ('${forbiddenPrefix}foo')") + } + assert(e2.getMessage.contains(forbiddenPrefix + "foo")) - val e3 = intercept[AnalysisException] { - sql(s"CREATE TABLE tbl TBLPROPERTIES ('${DATASOURCE_PREFIX}foo'='anything')") + val e3 = intercept[AnalysisException] { + sql(s"CREATE TABLE tbl TBLPROPERTIES ('${forbiddenPrefix}foo'='anything')") + } + assert(e3.getMessage.contains(forbiddenPrefix + "foo")) } - assert(e3.getMessage.contains(DATASOURCE_PREFIX + "foo")) } } } From 5d6e5599b558512000f3f62349276ebf19be366a Mon Sep 17 00:00:00 2001 From: Zhenhua Wang Date: Sat, 3 Sep 2016 09:27:09 +0800 Subject: [PATCH 25/25] add test cases --- .../command/AnalyzeTableCommand.scala | 5 +++-- .../spark/sql/hive/StatisticsSuite.scala | 18 ++++++++++++++++++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index d427cd219b9f..15687ddd728a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -103,8 +103,9 @@ case class AnalyzeTableCommand(tableName: String, noscan: Boolean = true) extend if (newTotalSize > 0 && newTotalSize != oldTotalSize) { newStats = Some(Statistics(sizeInBytes = newTotalSize)) } - // We only set rowCount when noscan is false, because otherwise we can't know whether the - // row count we get (`oldRowCount`) is valid or not. + // We only set rowCount when noscan is false, because otherwise: + // 1. when total size is not changed, we don't need to alter the table; + // 2. when total size is changed, `oldRowCount` becomes invalid. // This is to make sure that we only record the right statistics. if (!noscan) { val newRowCount = Dataset.ofRows(sparkSession, relation).count() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 5ffb9b16fe82..33ed67575486 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { @@ -220,6 +221,11 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils checkMetastoreRelationStats(textTable, expectedStats = Some(Statistics(sizeInBytes = 5812, rowCount = Some(500)))) + sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") + // when the total size is not changed, the old row count is kept + checkMetastoreRelationStats(textTable, expectedStats = + Some(Statistics(sizeInBytes = 5812, rowCount = Some(500)))) + sql(s"INSERT INTO TABLE $textTable SELECT * FROM src") sql(s"ANALYZE TABLE $textTable COMPUTE STATISTICS noscan") // update total size and remove the old and invalid row count @@ -301,6 +307,18 @@ class StatisticsSuite extends QueryTest with TestHiveSingleton with SQLTestUtils } } + test("statistics collection of a table with zero column") { + val table_no_cols = "table_no_cols" + withTable(table_no_cols) { + val rddNoCols = sparkContext.parallelize(1 to 10).map(_ => Row.empty) + val dfNoCols = spark.createDataFrame(rddNoCols, StructType(Seq.empty)) + dfNoCols.write.format("json").saveAsTable(table_no_cols) + sql(s"ANALYZE TABLE $table_no_cols COMPUTE STATISTICS") + checkLogicalRelationStats(table_no_cols, expectedStats = + Some(Statistics(sizeInBytes = 30, rowCount = Some(10)))) + } + } + test("estimates the size of a test MetastoreRelation") { val df = sql("""SELECT * FROM src""") val sizes = df.queryExecution.analyzed.collect { case mr: MetastoreRelation =>