From 2ca2c38643d648f48638eb90b2a17b099047ce70 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 3 Jun 2016 15:34:02 +0800 Subject: [PATCH 01/45] init commit --- .../spark/sql/execution/ExistingRDD.scala | 6 +- .../datasources/FileSourceStrategy.scala | 194 ++++++++++-------- .../apache/spark/sql/internal/SQLConf.scala | 7 + .../datasources/FileSourceStrategySuite.scala | 15 ++ 4 files changed, 130 insertions(+), 92 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index fef3255c739a..1998497f8bd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -338,7 +338,8 @@ private[sql] object DataSourceScanExec { rdd: RDD[InternalRow], relation: BaseRelation, metadata: Map[String, String] = Map.empty, - metastoreTableIdentifier: Option[TableIdentifier] = None): DataSourceScanExec = { + metastoreTableIdentifier: Option[TableIdentifier] = None, + isSupportBatch: Boolean = true): DataSourceScanExec = { val outputPartitioning = { val bucketSpec = relation match { // TODO: this should be closer to bucket planning. @@ -362,7 +363,8 @@ private[sql] object DataSourceScanExec { relation match { case r: HadoopFsRelation - if r.fileFormat.supportBatch(r.sparkSession, StructType.fromAttributes(output)) => + if isSupportBatch && + r.fileFormat.supportBatch(r.sparkSession, StructType.fromAttributes(output)) => BatchedDataSourceScanExec( output, rdd, relation, outputPartitioning, metadata, metastoreTableIdentifier) case _ => 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 350508c1d9f4..545821c6c168 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 @@ -109,108 +109,43 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}") - val readFile = files.fileFormat.buildReaderWithPartitionValues( - sparkSession = files.sparkSession, - dataSchema = files.dataSchema, - partitionSchema = files.partitionSchema, - requiredSchema = prunedDataSchema, - filters = pushedDownFilters, - options = files.options, - hadoopConf = files.sparkSession.sessionState.newHadoopConfWithOptions(files.options)) - - val plannedPartitions = files.bucketSpec match { - case Some(bucketing) if files.sparkSession.sessionState.conf.bucketingEnabled => - logInfo(s"Planning with ${bucketing.numBuckets} buckets") - val bucketed = - selectedPartitions.flatMap { p => - p.files.map { f => - val hosts = getBlockHosts(getBlockLocations(f), 0, f.getLen) - PartitionedFile(p.values, f.getPath.toUri.toString, 0, f.getLen, hosts) - } - }.groupBy { f => - BucketingUtils - .getBucketId(new Path(f.filePath).getName) - .getOrElse(sys.error(s"Invalid bucket file ${f.filePath}")) - } - - (0 until bucketing.numBuckets).map { bucketId => - FilePartition(bucketId, bucketed.getOrElse(bucketId, Nil)) - } - - case _ => - val defaultMaxSplitBytes = files.sparkSession.sessionState.conf.filesMaxPartitionBytes - val openCostInBytes = files.sparkSession.sessionState.conf.filesOpenCostInBytes - val defaultParallelism = files.sparkSession.sparkContext.defaultParallelism - val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum - val bytesPerCore = totalBytes / defaultParallelism - val maxSplitBytes = Math.min(defaultMaxSplitBytes, - Math.max(openCostInBytes, bytesPerCore)) - logInfo(s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + - s"open cost is considered as scanning $openCostInBytes bytes.") - - val splitFiles = selectedPartitions.flatMap { partition => - partition.files.flatMap { file => - val blockLocations = getBlockLocations(file) - (0L until file.getLen by maxSplitBytes).map { offset => - val remaining = file.getLen - offset - val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining - val hosts = getBlockHosts(blockLocations, offset, size) - PartitionedFile(partition.values, file.getPath.toUri.toString, offset, size, hosts) - } - } - }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) - - val partitions = new ArrayBuffer[FilePartition] - val currentFiles = new ArrayBuffer[PartitionedFile] - var currentSize = 0L - - /** Add the given file to the current partition. */ - def addFile(file: PartitionedFile): Unit = { - currentSize += file.length + openCostInBytes - currentFiles.append(file) - } - - /** Close the current partition and move to the next. */ - def closePartition(): Unit = { - if (currentFiles.nonEmpty) { - val newPartition = - FilePartition( - partitions.size, - currentFiles.toArray.toSeq) // Copy to a new Array. - partitions.append(newPartition) - } - currentFiles.clear() - currentSize = 0 - } - - // Assign files to partitions using "First Fit Decreasing" (FFD) - // TODO: consider adding a slop factor here? - splitFiles.foreach { file => - if (currentSize + file.length > maxSplitBytes) { - closePartition() - } - addFile(file) - } - closePartition() - partitions - } + val optimizerMetadataOnly = + readDataColumns.isEmpty && files.sparkSession.sessionState.conf.optimizerMetadataOnly + val scanRdd = if (optimizerMetadataOnly) { + val partitionValues = selectedPartitions.map(_.values) + files.sqlContext.sparkContext.parallelize(partitionValues, 1) + } else { + val readFile = files.fileFormat.buildReaderWithPartitionValues( + sparkSession = files.sparkSession, + dataSchema = files.dataSchema, + partitionSchema = files.partitionSchema, + requiredSchema = prunedDataSchema, + filters = pushedDownFilters, + options = files.options, + hadoopConf = files.sparkSession.sessionState.newHadoopConfWithOptions(files.options)) + + val plannedPartitions = getFilePartitions(files, selectedPartitions) + new FileScanRDD( + files.sparkSession, + readFile, + plannedPartitions) + } val meta = Map( "Format" -> files.fileFormat.toString, "ReadSchema" -> prunedDataSchema.simpleString, + "metadataOnly" -> optimizerMetadataOnly.toString, PUSHED_FILTERS -> pushedDownFilters.mkString("[", ", ", "]"), INPUT_PATHS -> files.location.paths.mkString(", ")) val scan = DataSourceScanExec.create( readDataColumns ++ partitionColumns, - new FileScanRDD( - files.sparkSession, - readFile, - plannedPartitions), + scanRdd, files, meta, - table) + table, + !optimizerMetadataOnly) val afterScanFilter = afterScanFilters.toSeq.reduceOption(expressions.And) val withFilter = afterScanFilter.map(execution.FilterExec(_, scan)).getOrElse(scan) @@ -225,6 +160,85 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { case _ => Nil } + private def getFilePartitions( + files: HadoopFsRelation, + selectedPartitions: Seq[Partition]): Seq[FilePartition] = files.bucketSpec match { + case Some(bucketing) if files.sparkSession.sessionState.conf.bucketingEnabled => + logInfo(s"Planning with ${bucketing.numBuckets} buckets") + val bucketed = + selectedPartitions.flatMap { p => + p.files.map { f => + val hosts = getBlockHosts(getBlockLocations(f), 0, f.getLen) + PartitionedFile(p.values, f.getPath.toUri.toString, 0, f.getLen, hosts) + } + }.groupBy { f => + BucketingUtils + .getBucketId(new Path(f.filePath).getName) + .getOrElse(sys.error(s"Invalid bucket file ${f.filePath}")) + } + + (0 until bucketing.numBuckets).map { bucketId => + FilePartition(bucketId, bucketed.getOrElse(bucketId, Nil)) + } + + case _ => + val defaultMaxSplitBytes = files.sparkSession.sessionState.conf.filesMaxPartitionBytes + val openCostInBytes = files.sparkSession.sessionState.conf.filesOpenCostInBytes + val defaultParallelism = files.sparkSession.sparkContext.defaultParallelism + val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum + val bytesPerCore = totalBytes / defaultParallelism + val maxSplitBytes = Math.min(defaultMaxSplitBytes, + Math.max(openCostInBytes, bytesPerCore)) + logInfo(s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + + s"open cost is considered as scanning $openCostInBytes bytes.") + + val splitFiles = selectedPartitions.flatMap { partition => + partition.files.flatMap { file => + val blockLocations = getBlockLocations(file) + (0L until file.getLen by maxSplitBytes).map { offset => + val remaining = file.getLen - offset + val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining + val hosts = getBlockHosts(blockLocations, offset, size) + PartitionedFile(partition.values, file.getPath.toUri.toString, offset, size, hosts) + } + } + }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) + + val partitions = new ArrayBuffer[FilePartition] + val currentFiles = new ArrayBuffer[PartitionedFile] + var currentSize = 0L + + /** Add the given file to the current partition. */ + def addFile(file: PartitionedFile): Unit = { + currentSize += file.length + openCostInBytes + currentFiles.append(file) + } + + /** Close the current partition and move to the next. */ + def closePartition(): Unit = { + if (currentFiles.nonEmpty) { + val newPartition = + FilePartition( + partitions.size, + currentFiles.toArray.toSeq) // Copy to a new Array. + partitions.append(newPartition) + } + currentFiles.clear() + currentSize = 0 + } + + // Assign files to partitions using "First Fit Decreasing" (FFD) + // TODO: consider adding a slop factor here? + splitFiles.foreach { file => + if (currentSize + file.length > maxSplitBytes) { + closePartition() + } + addFile(file) + } + closePartition() + partitions + } + private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { case f: LocatedFileStatus => f.getBlockLocations case f => Array.empty[BlockLocation] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 437e093825f6..8789724a2f6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -258,6 +258,11 @@ object SQLConf { .booleanConf .createWithDefault(false) + val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") + .doc("When true, enable the metadata-only query optimization.") + .booleanConf + .createWithDefault(false) + val NATIVE_VIEW = SQLConfigBuilder("spark.sql.nativeView") .internal() .doc("When true, CREATE VIEW will be handled by Spark SQL instead of Hive native commands. " + @@ -599,6 +604,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def metastorePartitionPruning: Boolean = getConf(HIVE_METASTORE_PARTITION_PRUNING) + def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY) + def nativeView: Boolean = getConf(NATIVE_VIEW) def wholeStageEnabled: Boolean = getConf(WHOLESTAGE_CODEGEN_ENABLED) 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 52dda8c6ace8..15d14bd9ba38 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 @@ -338,6 +338,21 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi assert(partitions.flatMap(fileScanRDD.preferredLocations).length == 3) } } + + test("optimize metadataOnly") { + withSQLConf("spark.sql.optimizer.metadataOnly" -> "true") { + val table = + createTable( + files = Seq( + "p1=1/file1" -> 10, + "p1=2/file2" -> 10)) + + checkDataset(table.select($"p1"), Row(1), Row(2)) + checkDataset(table.where("p1 = 1").select($"p1"), Row(1)) + val df = table.where("p1 = 1 AND (p1 + c1) = 2 AND c1 = 1") + assert(getPhysicalFilters(df) contains resolve(df, "c1 = 1")) + } + } } // Helpers for checking the arguments passed to the FileFormat. From edea710f826477de000913b58568cf9fd4b55814 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 3 Jun 2016 20:58:48 +0800 Subject: [PATCH 02/45] fix unit test --- .../datasources/FileSourceStrategySuite.scala | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) 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 15d14bd9ba38..c2a617080bbe 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 @@ -338,20 +338,20 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi assert(partitions.flatMap(fileScanRDD.preferredLocations).length == 3) } } + } - test("optimize metadataOnly") { - withSQLConf("spark.sql.optimizer.metadataOnly" -> "true") { - val table = - createTable( - files = Seq( - "p1=1/file1" -> 10, - "p1=2/file2" -> 10)) - - checkDataset(table.select($"p1"), Row(1), Row(2)) - checkDataset(table.where("p1 = 1").select($"p1"), Row(1)) - val df = table.where("p1 = 1 AND (p1 + c1) = 2 AND c1 = 1") - assert(getPhysicalFilters(df) contains resolve(df, "c1 = 1")) - } + test("optimize metadataOnly") { + withSQLConf("spark.sql.optimizer.metadataOnly" -> "true") { + val table = + createTable( + files = Seq( + "p1=1/file1" -> 10, + "p1=2/file2" -> 10)) + + checkDataset(table.select($"p1"), Row(1), Row(2)) + checkDataset(table.where("p1 = 1").select($"p1"), Row(1)) + val df = table.where("p1 = 1 AND (p1 + c1) = 2 AND c1 = 1") + assert(getPhysicalFilters(df) contains resolve(df, "c1 = 1")) } } From 153293ea976f78b0b55d6744a66ecab7e8bedb62 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Sat, 4 Jun 2016 00:57:52 +0800 Subject: [PATCH 03/45] fix unit test --- .../datasources/FileSourceStrategy.scala | 44 ++++++++++--------- 1 file changed, 24 insertions(+), 20 deletions(-) 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 545821c6c168..75a4c31cbc7a 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 @@ -22,9 +22,11 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.{expressions, InternalRow} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.DataSourceScanExec @@ -111,25 +113,27 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { val optimizerMetadataOnly = readDataColumns.isEmpty && files.sparkSession.sessionState.conf.optimizerMetadataOnly - val scanRdd = if (optimizerMetadataOnly) { - val partitionValues = selectedPartitions.map(_.values) - files.sqlContext.sparkContext.parallelize(partitionValues, 1) - } else { - val readFile = files.fileFormat.buildReaderWithPartitionValues( - sparkSession = files.sparkSession, - dataSchema = files.dataSchema, - partitionSchema = files.partitionSchema, - requiredSchema = prunedDataSchema, - filters = pushedDownFilters, - options = files.options, - hadoopConf = files.sparkSession.sessionState.newHadoopConfWithOptions(files.options)) - - val plannedPartitions = getFilePartitions(files, selectedPartitions) - new FileScanRDD( - files.sparkSession, - readFile, - plannedPartitions) - } + val scanRdd: RDD[InternalRow] = if (optimizerMetadataOnly) { + val partitionSchema = files.partitionSchema.toAttributes + lazy val converter = GenerateUnsafeProjection.generate(partitionSchema, partitionSchema) + val partitionValues = selectedPartitions.map(_.values) + files.sqlContext.sparkContext.parallelize(partitionValues, 1).map(converter(_)) + } else { + val readFile = files.fileFormat.buildReaderWithPartitionValues( + sparkSession = files.sparkSession, + dataSchema = files.dataSchema, + partitionSchema = files.partitionSchema, + requiredSchema = prunedDataSchema, + filters = pushedDownFilters, + options = files.options, + hadoopConf = files.sparkSession.sessionState.newHadoopConfWithOptions(files.options)) + + val plannedPartitions = getFilePartitions(files, selectedPartitions) + new FileScanRDD( + files.sparkSession, + readFile, + plannedPartitions) + } val meta = Map( "Format" -> files.fileFormat.toString, From 7dfb7437a0918c79a7856b35f06dd6edfe63f08d Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 24 Jun 2016 14:29:32 +0800 Subject: [PATCH 04/45] update --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 115 ++++++++++++++++++ .../spark/sql/hive/HiveSessionCatalog.scala | 1 + .../spark/sql/hive/HiveSessionState.scala | 15 ++- .../sql/hive/execution/SQLQuerySuite.scala | 40 ++++++ 4 files changed, 170 insertions(+), 1 deletion(-) 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 f10afa75f2bf..6dc028f01a62 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 @@ -27,9 +27,11 @@ import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.command.CreateHiveTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} @@ -506,6 +508,119 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } } + /** + * When scanning only partition columns, get results based on metadata without scanning files. + * It is used for distinct or distinct/Max/Min aggregations, example: max(partition). + */ + object MetadataOnlyOptimizer extends Rule[LogicalPlan] { + + private def canSupportMetadataOnly(a: Aggregate): Boolean = { + val aggregateExpressions = a.aggregateExpressions.flatMap { expr => + expr.collect { + case agg: AggregateExpression => agg + } + }.distinct + aggregateExpressions.forall { agg => + if (agg.isDistinct) { + true + } else { + agg.aggregateFunction match { + case max: Max => true + case min: Min => true + case _ => false + } + } + } + } + + private def findRelation(plan: LogicalPlan): (Option[LogicalPlan], Seq[Expression]) = { + plan match { + case relation @ LogicalRelation(files: HadoopFsRelation, _, table) + if files.partitionSchema.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case relation: MetastoreRelation if relation.partitionKeys.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case p @ Project(_, child) => + findRelation(child) + + case f @ Filter(filterCondition, child) => + val (plan, conditions) = findRelation(child) + (plan, conditions ++ Seq(filterCondition)) + + case SubqueryAlias(_, child) => + findRelation(child) + + case _ => (None, Seq.empty[Expression]) + } + } + + private def convertToMetadataOnlyPlan( + parent: LogicalPlan, + filters: Seq[Expression], + relation: LogicalPlan): LogicalPlan = relation match { + case l @ LogicalRelation(files: HadoopFsRelation, _, _) => + val attributeMap = l.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = files.partitionSchema.map { field => + attributeMap.getOrElse(field.name, throw new AnalysisException( + s"Unable to resolve ${field.name} given [${l.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val selectedPartitions = files.location.listFiles(filters) + val partitionValues = selectedPartitions.map(_.values) + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + parent.withNewChildren(LogicalRDD(partitionColumns, valuesRdd)(sparkSession) :: Nil) + } else { + parent + } + + case relation: MetastoreRelation => + if (parent.references.subsetOf(AttributeSet(relation.partitionKeys))) { + val partitionColumnDataTypes = relation.partitionKeys.map(_.dataType) + val partitionValues = relation.getHiveQlPartitions(filters).map { p => + InternalRow.fromSeq(p.getValues.asScala.zip(partitionColumnDataTypes).map { + case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) + }) + } + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(relation.partitionKeys, valuesRdd)(sparkSession) + val child = filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) + parent.withNewChildren(child :: Nil) + } else { + parent + } + + case _ => + parent + } + + def apply(plan: LogicalPlan): LogicalPlan = { + if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { + return plan + } + plan.transform { + case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => + val (plan, filters) = findRelation(child) + if (plan.isDefined) { + convertToMetadataOnlyPlan(a, filters, plan.get) + } else { + a + } + + case d @ Distinct(p @ Project(_, _)) => + val (plan, filters) = findRelation(p) + if (plan.isDefined) { + d.withNewChildren(convertToMetadataOnlyPlan(p, filters, plan.get) :: Nil) + } else { + d + } + } + } + } + } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 4f8aac8c2fcd..8a19645480c1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -88,6 +88,7 @@ private[sql] class HiveSessionCatalog( val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions val CreateTables: Rule[LogicalPlan] = metastoreCatalog.CreateTables val PreInsertionCasts: Rule[LogicalPlan] = metastoreCatalog.PreInsertionCasts + val metadataOnlyOptimizer: Rule[LogicalPlan] = metastoreCatalog.MetadataOnlyOptimizer override def refreshTable(name: TableIdentifier): Unit = { metastoreCatalog.refreshTable(name) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index ca8e5f822396..1150414ffd78 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.hive import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Analyzer -import org.apache.spark.sql.execution.SparkPlanner +import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.execution.{SparkOptimizer, SparkPlanner} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.SessionState @@ -74,6 +75,18 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) } } + /** + * Logical query plan optimizer for Hive. + */ + override lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods) { + val extendedHiveOptimizerRules: Seq[Batch] = Seq( + Batch("MetadataOnly Optimization", Once, + catalog.metadataOnlyOptimizer) + ) + + override def batches: Seq[Batch] = extendedHiveOptimizerRules ++ super.batches + } + /** * Planner that takes into account Hive-specific strategies. */ 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 499819f32b43..b1aa17de3a25 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 @@ -1640,4 +1640,44 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ) } } + + test("spark-15752 metadata only optimizer") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + spark.range(0, 5, 1, 2).selectExpr("id as key", "id as value").registerTempTable("tempTable") + sql( + """ + |CREATE TABLE srcpart_15752 (key INT, value STRING) + |PARTITIONED BY (ds STRING, hr INT) STORED AS parquet + """.stripMargin) + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { + sql( + s""" + |INSERT OVERWRITE TABLE srcpart_15752 PARTITION (ds='$ds',hr='$hr') + |select key, value from tempTable + """.stripMargin) + } + checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) + checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) + checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) + checkAnswer(sql("select distinct hr from srcpart_15752 where hr = 11"), Row(11)) + + sql( + """ + |CREATE TABLE srctext_15752 (key INT, value STRING) + |PARTITIONED BY (ds STRING, hr INT) STORED AS textfile + """.stripMargin) + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { + sql( + s""" + |INSERT OVERWRITE TABLE srctext_15752 PARTITION (ds='$ds',hr='$hr') + |select key, value from tempTable + """.stripMargin) + } + + checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) + checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) + checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) + checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) + } + } } From 68e6d6d32a0b5f9f8503cc54ee491abe98db76dc Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 24 Jun 2016 14:30:50 +0800 Subject: [PATCH 05/45] Revert "fix unit test" This reverts commit 153293ea976f78b0b55d6744a66ecab7e8bedb62. --- .../datasources/FileSourceStrategy.scala | 44 +++++++++---------- 1 file changed, 20 insertions(+), 24 deletions(-) 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 75a4c31cbc7a..545821c6c168 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 @@ -22,11 +22,9 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{expressions, InternalRow} +import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.DataSourceScanExec @@ -113,27 +111,25 @@ private[sql] object FileSourceStrategy extends Strategy with Logging { val optimizerMetadataOnly = readDataColumns.isEmpty && files.sparkSession.sessionState.conf.optimizerMetadataOnly - val scanRdd: RDD[InternalRow] = if (optimizerMetadataOnly) { - val partitionSchema = files.partitionSchema.toAttributes - lazy val converter = GenerateUnsafeProjection.generate(partitionSchema, partitionSchema) - val partitionValues = selectedPartitions.map(_.values) - files.sqlContext.sparkContext.parallelize(partitionValues, 1).map(converter(_)) - } else { - val readFile = files.fileFormat.buildReaderWithPartitionValues( - sparkSession = files.sparkSession, - dataSchema = files.dataSchema, - partitionSchema = files.partitionSchema, - requiredSchema = prunedDataSchema, - filters = pushedDownFilters, - options = files.options, - hadoopConf = files.sparkSession.sessionState.newHadoopConfWithOptions(files.options)) - - val plannedPartitions = getFilePartitions(files, selectedPartitions) - new FileScanRDD( - files.sparkSession, - readFile, - plannedPartitions) - } + val scanRdd = if (optimizerMetadataOnly) { + val partitionValues = selectedPartitions.map(_.values) + files.sqlContext.sparkContext.parallelize(partitionValues, 1) + } else { + val readFile = files.fileFormat.buildReaderWithPartitionValues( + sparkSession = files.sparkSession, + dataSchema = files.dataSchema, + partitionSchema = files.partitionSchema, + requiredSchema = prunedDataSchema, + filters = pushedDownFilters, + options = files.options, + hadoopConf = files.sparkSession.sessionState.newHadoopConfWithOptions(files.options)) + + val plannedPartitions = getFilePartitions(files, selectedPartitions) + new FileScanRDD( + files.sparkSession, + readFile, + plannedPartitions) + } val meta = Map( "Format" -> files.fileFormat.toString, From 595ef36ede338936727e18b0a84a9cc8280a0e29 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 24 Jun 2016 14:31:14 +0800 Subject: [PATCH 06/45] Revert "fix unit test" This reverts commit edea710f826477de000913b58568cf9fd4b55814. --- .../datasources/FileSourceStrategySuite.scala | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) 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 bab5b90b6359..645ef42ca4a9 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 @@ -338,20 +338,20 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi assert(partitions.flatMap(fileScanRDD.preferredLocations).length == 3) } } - } - - test("optimize metadataOnly") { - withSQLConf("spark.sql.optimizer.metadataOnly" -> "true") { - val table = - createTable( - files = Seq( - "p1=1/file1" -> 10, - "p1=2/file2" -> 10)) - checkDataset(table.select($"p1"), Row(1), Row(2)) - checkDataset(table.where("p1 = 1").select($"p1"), Row(1)) - val df = table.where("p1 = 1 AND (p1 + c1) = 2 AND c1 = 1") - assert(getPhysicalFilters(df) contains resolve(df, "c1 = 1")) + test("optimize metadataOnly") { + withSQLConf("spark.sql.optimizer.metadataOnly" -> "true") { + val table = + createTable( + files = Seq( + "p1=1/file1" -> 10, + "p1=2/file2" -> 10)) + + checkDataset(table.select($"p1"), Row(1), Row(2)) + checkDataset(table.where("p1 = 1").select($"p1"), Row(1)) + val df = table.where("p1 = 1 AND (p1 + c1) = 2 AND c1 = 1") + assert(getPhysicalFilters(df) contains resolve(df, "c1 = 1")) + } } } From 2e55a9df94ef85345f54f2bb3b8c23d06f600f58 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 24 Jun 2016 15:50:56 +0800 Subject: [PATCH 07/45] Merge branch 'apache-master' into metadata-only --- .../spark/sql/execution/ExistingRDD.scala | 6 +- .../datasources/FileSourceStrategySuite.scala | 15 -- .../spark/sql/hive/HiveMetastoreCatalog.scala | 153 +++++++++++++----- .../spark/sql/hive/HiveSessionCatalog.scala | 1 - .../sql/hive/execution/SQLQuerySuite.scala | 9 +- 5 files changed, 120 insertions(+), 64 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 75fad506d468..e2c23a4ba867 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -347,8 +347,7 @@ private[sql] object DataSourceScanExec { rdd: RDD[InternalRow], relation: BaseRelation, metadata: Map[String, String] = Map.empty, - metastoreTableIdentifier: Option[TableIdentifier] = None, - isSupportBatch: Boolean = true): DataSourceScanExec = { + metastoreTableIdentifier: Option[TableIdentifier] = None): DataSourceScanExec = { val outputPartitioning = { val bucketSpec = relation match { // TODO: this should be closer to bucket planning. @@ -372,8 +371,7 @@ private[sql] object DataSourceScanExec { relation match { case r: HadoopFsRelation - if isSupportBatch && - r.fileFormat.supportBatch(r.sparkSession, StructType.fromAttributes(output)) => + if r.fileFormat.supportBatch(r.sparkSession, StructType.fromAttributes(output)) => BatchedDataSourceScanExec( output, rdd, relation, outputPartitioning, metadata, metastoreTableIdentifier) case _ => 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 9a4e2859a973..8d8a18fa9332 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 @@ -338,21 +338,6 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi assert(partitions.flatMap(fileScanRDD.preferredLocations).length == 3) } } - - test("optimize metadataOnly") { - withSQLConf("spark.sql.optimizer.metadataOnly" -> "true") { - val table = - createTable( - files = Seq( - "p1=1/file1" -> 10, - "p1=2/file2" -> 10)) - - checkDataset(table.select($"p1"), Row(1), Row(2)) - checkDataset(table.where("p1 = 1").select($"p1"), Row(1)) - val df = table.where("p1 = 1 AND (p1 + c1) = 2 AND c1 = 1") - assert(getPhysicalFilters(df) contains resolve(df, "c1 = 1")) - } - } } test("SPARK-15654 do not split non-splittable files") { 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 f10afa75f2bf..a1e59d1996fc 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 @@ -23,21 +23,22 @@ import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.command.CreateHiveTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ - /** * Legacy catalog for interacting with the Hive metastore. * @@ -355,13 +356,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val fileFormatClass = classOf[ParquetFileFormat] val mergeSchema = sessionState.convertMetastoreParquetWithSchemaMerging - val options = Map( - ParquetFileFormat.MERGE_SCHEMA -> mergeSchema.toString, - ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier( - relation.tableName, - Some(relation.databaseName) - ).unquotedString - ) + val options = Map(ParquetOptions.MERGE_SCHEMA -> mergeSchema.toString) convertToLogicalRelation(relation, options, defaultSource, fileFormatClass, "parquet") } @@ -465,43 +460,119 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } /** - * Casts input data to correct data types according to table definition before inserting into - * that table. + * When scanning only partition columns, get results based on metadata without scanning files. + * It is used for distinct or distinct/Max/Min aggregations, example: max(partition). */ - object PreInsertionCasts extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.transform { - // Wait until children are resolved. - case p: LogicalPlan if !p.childrenResolved => p + object MetadataOnlyOptimizer extends Rule[LogicalPlan] { - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _, _) => - castChildOutput(p, table, child) + private def canSupportMetadataOnly(a: Aggregate): Boolean = { + val aggregateExpressions = a.aggregateExpressions.flatMap { expr => + expr.collect { + case agg: AggregateExpression => agg + } + }.distinct + aggregateExpressions.forall { agg => + if (agg.isDistinct) { + true + } else { + agg.aggregateFunction match { + case max: Max => true + case min: Min => true + case _ => false + } + } + } } - def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) - : LogicalPlan = { - val childOutputDataTypes = child.output.map(_.dataType) - val numDynamicPartitions = p.partition.values.count(_.isEmpty) - val tableOutputDataTypes = - (table.attributes ++ table.partitionKeys.takeRight(numDynamicPartitions)) - .take(child.output.length).map(_.dataType) - - if (childOutputDataTypes == tableOutputDataTypes) { - InsertIntoHiveTable(table, p.partition, p.child, p.overwrite, p.ifNotExists) - } else if (childOutputDataTypes.size == tableOutputDataTypes.size && - childOutputDataTypes.zip(tableOutputDataTypes) - .forall { case (left, right) => left.sameType(right) }) { - // If both types ignoring nullability of ArrayType, MapType, StructType are the same, - // use InsertIntoHiveTable instead of InsertIntoTable. - InsertIntoHiveTable(table, p.partition, p.child, p.overwrite, p.ifNotExists) - } else { - // Only do the casting when child output data types differ from table output data types. - val castedChildOutput = child.output.zip(table.output).map { - case (input, output) if input.dataType != output.dataType => - Alias(Cast(input, output.dataType), input.name)() - case (input, _) => input + private def findRelation(plan: LogicalPlan): (Option[LogicalPlan], Seq[Expression]) = { + plan match { + case relation @ LogicalRelation(files: HadoopFsRelation, _, table) + if files.partitionSchema.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case relation: MetastoreRelation if relation.partitionKeys.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case p @ Project(_, child) => + findRelation(child) + + case f @ Filter(filterCondition, child) => + val (plan, conditions) = findRelation(child) + (plan, conditions ++ Seq(filterCondition)) + + case SubqueryAlias(_, child) => + findRelation(child) + + case _ => (None, Seq.empty[Expression]) + } + } + + private def convertToMetadataOnlyPlan( + parent: LogicalPlan, + project: Option[LogicalPlan], + filters: Seq[Expression], + relation: LogicalPlan): LogicalPlan = relation match { + case l @ LogicalRelation(files: HadoopFsRelation, _, _) => + val attributeMap = l.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = files.partitionSchema.map { field => + attributeMap.getOrElse(field.name, throw new AnalysisException( + s"Unable to resolve ${field.name} given [${l.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val selectedPartitions = files.location.listFiles(filters) + val partitionValues = selectedPartitions.map(_.values) + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + val scanPlan = project.map(_.withNewChildren(valuesPlan :: Nil)).getOrElse(valuesPlan) + parent.withNewChildren(scanPlan :: Nil) + } else { + parent + } + + case relation: MetastoreRelation => + if (parent.references.subsetOf(AttributeSet(relation.partitionKeys))) { + val partitionColumnDataTypes = relation.partitionKeys.map(_.dataType) + val partitionValues = relation.getHiveQlPartitions(filters).map { p => + InternalRow.fromSeq(p.getValues.asScala.zip(partitionColumnDataTypes).map { + case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) + }) + } + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(relation.partitionKeys, valuesRdd)(sparkSession) + val filterPlan = + filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) + val scanPlan = project.map(_.withNewChildren(filterPlan :: Nil)).getOrElse(filterPlan) + parent.withNewChildren(scanPlan :: Nil) + } else { + parent } - p.copy(child = logical.Project(castedChildOutput, child)) + case _ => + parent + } + + def apply(plan: LogicalPlan): LogicalPlan = { + if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { + return plan + } + plan.transform { + case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => + val (plan, filters) = findRelation(child) + if (plan.isDefined) { + convertToMetadataOnlyPlan(a, None, filters, plan.get) + } else { + a + } + + case d @ Distinct(p @ Project(_, _)) => + val (plan, filters) = findRelation(p) + if (plan.isDefined) { + convertToMetadataOnlyPlan(d, Some(p), filters, plan.get) + } else { + d + } } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 8a19645480c1..c3b2a8c7d392 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -87,7 +87,6 @@ private[sql] class HiveSessionCatalog( val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions val CreateTables: Rule[LogicalPlan] = metastoreCatalog.CreateTables - val PreInsertionCasts: Rule[LogicalPlan] = metastoreCatalog.PreInsertionCasts val metadataOnlyOptimizer: Rule[LogicalPlan] = metastoreCatalog.MetadataOnlyOptimizer override def refreshTable(name: TableIdentifier): Unit = { 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 8c260bb6560b..37b312da7763 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 @@ -1687,7 +1687,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("spark-15752 metadata only optimizer") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - spark.range(0, 5, 1, 2).selectExpr("id as key", "id as value").registerTempTable("tempTable") + val df = Seq((1, 2), (3, 4)).toDF("key", "value") + df.createOrReplaceTempView("data") sql( """ |CREATE TABLE srcpart_15752 (key INT, value STRING) @@ -1697,12 +1698,13 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql( s""" |INSERT OVERWRITE TABLE srcpart_15752 PARTITION (ds='$ds',hr='$hr') - |select key, value from tempTable + |select key, value from data """.stripMargin) } checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) + checkAnswer(sql("select distinct hr from srcpart_15752"), Row(11) :: Row(12) :: Nil) checkAnswer(sql("select distinct hr from srcpart_15752 where hr = 11"), Row(11)) sql( @@ -1714,13 +1716,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql( s""" |INSERT OVERWRITE TABLE srctext_15752 PARTITION (ds='$ds',hr='$hr') - |select key, value from tempTable + |select key, value from data """.stripMargin) } checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) + checkAnswer(sql("select distinct hr from srctext_15752"), Row(11) :: Row(12) :: Nil) checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) } } From b2b6eba2ad21675173227ee19c65f6661534e6b6 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 24 Jun 2016 15:52:21 +0800 Subject: [PATCH 08/45] update --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 a1e59d1996fc..c4f478ba4c25 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 @@ -621,7 +621,7 @@ private[hive] case class InsertIntoHiveTable( child: LogicalPlan, overwrite: Boolean, ifNotExists: Boolean) - extends LogicalPlan { + extends LogicalPlan with Command { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty From 6404c1fa2e8e9796574b69b57054dd4791ff6c52 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Sat, 25 Jun 2016 00:12:32 +0800 Subject: [PATCH 09/45] update opt for core --- .../catalyst/catalog/ExternalCatalog.scala | 14 ++ .../catalyst/catalog/InMemoryCatalog.scala | 8 + .../sql/catalyst/catalog/SessionCatalog.scala | 13 ++ .../sql/catalyst/optimizer/Optimizer.scala | 4 + .../spark/sql/execution/SparkOptimizer.scala | 152 +++++++++++++++++- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../spark/sql/hive/HiveExternalCatalog.scala | 10 ++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 121 -------------- .../spark/sql/hive/HiveSessionCatalog.scala | 1 - .../spark/sql/hive/HiveSessionState.scala | 12 -- .../spark/sql/hive/client/HiveClient.scala | 11 ++ .../sql/hive/execution/SQLQuerySuite.scala | 8 +- 12 files changed, 214 insertions(+), 142 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 6714846e8cbd..5d478a4ccce9 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +import org.apache.spark.sql.catalyst.expressions.Expression /** @@ -151,6 +152,19 @@ abstract class ExternalCatalog { def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition + /** + * Returns partitions filtered by predicates for the given table, It just work for Hive. + * + * The filters Expressions may optionally be provided to filter the partitions returned. + * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'), + * then the filters (a='1') will return the first two only. + * @param catalogTable table + * @param filters The filters used to prune which partitions are returned. + */ + def getPartitionsByFilter( + catalogTable: CatalogTable, + filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] + /** * List the metadata of all partitions that belong to the specified table, assuming it exists. * 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 fb3e1b3637f2..1d9b30d3f873 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 @@ -28,6 +28,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils /** @@ -447,6 +448,13 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E catalog(db).tables(table).partitions(spec) } + override def getPartitionsByFilter( + catalogTable: CatalogTable, + filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] = synchronized { + requireTableExists(catalogTable.database, catalogTable.identifier.table) + catalog(catalogTable.database).tables(catalogTable.identifier.table).partitions.values.toSeq + } + override def listPartitions( db: String, table: String, 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 7ab10d1c3823..be984bade600 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 @@ -585,6 +585,19 @@ class SessionCatalog( externalCatalog.getPartition(db, table, spec) } + /** + * Returns partitions filtered by predicates for the given table, It just work for Hive. + * + * The filters Expressions may optionally be provided to filter the partitions returned. + * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'), + * then the filters (a='1') will return the first two only. + */ + def getPartitionsByFilter( + catalogTable: CatalogTable, + filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] = { + externalCatalog.getPartitionsByFilter(catalogTable, filters) + } + /** * List the metadata of all partitions that belong to the specified table, assuming it exists. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 2bca31d5f1f7..3b567571a07c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -42,6 +42,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) protected val fixedPoint = FixedPoint(conf.optimizerMaxIterations) + val aggregateOptimizationsRules: Seq[Rule[LogicalPlan]] = RewriteDistinctAggregates :: Nil + def batches: Seq[Batch] = { // Technically some of the rules in Finish Analysis are not optimizer rules and belong more // in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime). @@ -61,6 +63,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // extra operators between two adjacent Union operators. // - Call CombineUnions again in Batch("Operator Optimizations"), // since the other rules might make two separate Unions operators adjacent. + Batch("Aggregate Optimizations", Once, + aggregateOptimizationsRules : _*) :: Batch("Union", Once, CombineUnions) :: Batch("Subquery", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 12a10cba20fe..e4ad530c4d3f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -17,9 +17,16 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.ExperimentalMethods -import org.apache.spark.sql.catalyst.catalog.SessionCatalog -import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.{AnalysisException, ExperimentalMethods, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, SessionCatalog} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.optimizer._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate import org.apache.spark.sql.internal.SQLConf @@ -29,7 +36,146 @@ class SparkOptimizer( experimentalMethods: ExperimentalMethods) extends Optimizer(catalog, conf) { + override val aggregateOptimizationsRules: Seq[Rule[LogicalPlan]] = + MetadataOnlyOptimizer(catalog) :: + RewriteDistinctAggregates :: Nil + override def batches: Seq[Batch] = super.batches :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } + +/** + * When scanning only partition columns, get results based on metadata without scanning files. + * It is used for distinct, distinct aggregations or distinct-like aggregations(example: Max/Min). + * Example: select Max(partition) from table. + */ +case class MetadataOnlyOptimizer(catalog: SessionCatalog) extends Rule[LogicalPlan] { + + private val sparkSession: SparkSession = SparkSession.getActiveSession.get + private def canSupportMetadataOnly(a: Aggregate): Boolean = { + val aggregateExpressions = a.aggregateExpressions.flatMap { expr => + expr.collect { + case agg: AggregateExpression => agg + } + }.distinct + if (aggregateExpressions.isEmpty) { + // Cannot support for aggregate that has no aggregateFunction. + // example: select col1 from table group by col1. + false + } else { + aggregateExpressions.forall { agg => + if (agg.isDistinct) { + true + } else { + // If function can be evaluated on just the distinct values of a column, it can be used + // by metadata-only optimizer. + agg.aggregateFunction match { + case max: Max => true + case min: Min => true + case hyperLog: HyperLogLogPlusPlus => true + case _ => false + } + } + } + } + } + + private def findRelation(plan: LogicalPlan): (Option[LogicalPlan], Seq[Expression]) = { + plan match { + case relation @ LogicalRelation(files: HadoopFsRelation, _, table) + if files.partitionSchema.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case p @ Project(_, child) => + findRelation(child) + + case f @ Filter(filterCondition, child) => + val (plan, conditions) = findRelation(child) + (plan, conditions ++ Seq(filterCondition)) + + case _ => (None, Seq.empty[Expression]) + } + } + + private def convertToMetadataOnlyPlan( + parent: LogicalPlan, + project: Option[LogicalPlan], + filters: Seq[Expression], + relation: LogicalPlan): LogicalPlan = relation match { + case l @ LogicalRelation(files: HadoopFsRelation, _, _) => + val attributeMap = l.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = files.partitionSchema.map { field => + attributeMap.getOrElse(field.name, throw new AnalysisException( + s"Unable to resolve ${field.name} given [${l.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val selectedPartitions = files.location.listFiles(filters) + val partitionValues = selectedPartitions.map(_.values) + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + val scanPlan = project.map(_.withNewChildren(valuesPlan :: Nil)).getOrElse(valuesPlan) + parent.withNewChildren(scanPlan :: Nil) + } else { + parent + } + + case relation: CatalogRelation => + val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => + attributeMap.getOrElse(column, throw new AnalysisException( + s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val partitionColumnDataTypes = partitionColumns.map(_.dataType) + val partitionValues = catalog.getPartitionsByFilter(relation.catalogTable, filters) + .map { p => + InternalRow.fromSeq( + partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { + case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) + }) + } + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + val filterPlan = + filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) + val scanPlan = project.map(_.withNewChildren(filterPlan :: Nil)).getOrElse(filterPlan) + parent.withNewChildren(scanPlan :: Nil) + } else { + parent + } + + case _ => + parent + } + + def apply(plan: LogicalPlan): LogicalPlan = { + if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { + return plan + } + plan.transform { + case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => + val (plan, filters) = findRelation(child) + if (plan.isDefined) { + convertToMetadataOnlyPlan(a, None, filters, plan.get) + } else { + a + } + + case d @ Distinct(p @ Project(_, _)) => + val (plan, filters) = findRelation(p) + if (plan.isDefined) { + convertToMetadataOnlyPlan(d, Some(p), filters, plan.get) + } else { + d + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index f18ab6463f69..9ed640d6e6a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -261,7 +261,7 @@ object SQLConf { val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") .doc("When true, enable the metadata-only query optimization.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val NATIVE_VIEW = SQLConfigBuilder("spark.sql.nativeView") .internal() 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 b8bc9ab900ad..bfb53b915da8 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 @@ -30,6 +30,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.expressions.Expression import org.apache.spark.sql.hive.client.HiveClient @@ -322,6 +323,15 @@ private[spark] class HiveExternalCatalog(client: HiveClient, hadoopConf: Configu client.getPartition(db, table, spec) } + /** + * Returns partitions filtered by predicates for the given table. + */ + override def getPartitionsByFilter( + catalogTable: CatalogTable, + filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] = withClient { + client.getPartitionsByFilter(catalogTable, filters) + } + /** * Returns the partition names from hive metastore for a given table in a database. */ 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 c4f478ba4c25..3c7b70a4bd53 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 @@ -27,11 +27,8 @@ import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ import org.apache.spark.sql.execution.command.CreateHiveTableAsSelectLogicalPlan import org.apache.spark.sql.execution.datasources.{Partition => _, _} @@ -459,124 +456,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } } - /** - * When scanning only partition columns, get results based on metadata without scanning files. - * It is used for distinct or distinct/Max/Min aggregations, example: max(partition). - */ - object MetadataOnlyOptimizer extends Rule[LogicalPlan] { - - private def canSupportMetadataOnly(a: Aggregate): Boolean = { - val aggregateExpressions = a.aggregateExpressions.flatMap { expr => - expr.collect { - case agg: AggregateExpression => agg - } - }.distinct - aggregateExpressions.forall { agg => - if (agg.isDistinct) { - true - } else { - agg.aggregateFunction match { - case max: Max => true - case min: Min => true - case _ => false - } - } - } - } - - private def findRelation(plan: LogicalPlan): (Option[LogicalPlan], Seq[Expression]) = { - plan match { - case relation @ LogicalRelation(files: HadoopFsRelation, _, table) - if files.partitionSchema.nonEmpty => - (Some(relation), Seq.empty[Expression]) - - case relation: MetastoreRelation if relation.partitionKeys.nonEmpty => - (Some(relation), Seq.empty[Expression]) - - case p @ Project(_, child) => - findRelation(child) - - case f @ Filter(filterCondition, child) => - val (plan, conditions) = findRelation(child) - (plan, conditions ++ Seq(filterCondition)) - - case SubqueryAlias(_, child) => - findRelation(child) - - case _ => (None, Seq.empty[Expression]) - } - } - - private def convertToMetadataOnlyPlan( - parent: LogicalPlan, - project: Option[LogicalPlan], - filters: Seq[Expression], - relation: LogicalPlan): LogicalPlan = relation match { - case l @ LogicalRelation(files: HadoopFsRelation, _, _) => - val attributeMap = l.output.map(attr => (attr.name, attr)).toMap - val partitionColumns = files.partitionSchema.map { field => - attributeMap.getOrElse(field.name, throw new AnalysisException( - s"Unable to resolve ${field.name} given [${l.output.map(_.name).mkString(", ")}]")) - } - val filterColumns = filters.flatMap(_.references) - val projectSet = parent.references ++ AttributeSet(filterColumns) - if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val selectedPartitions = files.location.listFiles(filters) - val partitionValues = selectedPartitions.map(_.values) - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - val scanPlan = project.map(_.withNewChildren(valuesPlan :: Nil)).getOrElse(valuesPlan) - parent.withNewChildren(scanPlan :: Nil) - } else { - parent - } - - case relation: MetastoreRelation => - if (parent.references.subsetOf(AttributeSet(relation.partitionKeys))) { - val partitionColumnDataTypes = relation.partitionKeys.map(_.dataType) - val partitionValues = relation.getHiveQlPartitions(filters).map { p => - InternalRow.fromSeq(p.getValues.asScala.zip(partitionColumnDataTypes).map { - case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) - }) - } - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) - val valuesPlan = LogicalRDD(relation.partitionKeys, valuesRdd)(sparkSession) - val filterPlan = - filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) - val scanPlan = project.map(_.withNewChildren(filterPlan :: Nil)).getOrElse(filterPlan) - parent.withNewChildren(scanPlan :: Nil) - } else { - parent - } - - case _ => - parent - } - - def apply(plan: LogicalPlan): LogicalPlan = { - if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { - return plan - } - plan.transform { - case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => - val (plan, filters) = findRelation(child) - if (plan.isDefined) { - convertToMetadataOnlyPlan(a, None, filters, plan.get) - } else { - a - } - - case d @ Distinct(p @ Project(_, _)) => - val (plan, filters) = findRelation(p) - if (plan.isDefined) { - convertToMetadataOnlyPlan(d, Some(p), filters, plan.get) - } else { - d - } - } - } - } - } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index c3b2a8c7d392..2f6a2207855e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -87,7 +87,6 @@ private[sql] class HiveSessionCatalog( val ParquetConversions: Rule[LogicalPlan] = metastoreCatalog.ParquetConversions val OrcConversions: Rule[LogicalPlan] = metastoreCatalog.OrcConversions val CreateTables: Rule[LogicalPlan] = metastoreCatalog.CreateTables - val metadataOnlyOptimizer: Rule[LogicalPlan] = metastoreCatalog.MetadataOnlyOptimizer override def refreshTable(name: TableIdentifier): Unit = { metastoreCatalog.refreshTable(name) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 924b5a0b2a46..46978d5e64a2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -74,18 +74,6 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) } } - /** - * Logical query plan optimizer for Hive. - */ - override lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods) { - val extendedHiveOptimizerRules: Seq[Batch] = Seq( - Batch("MetadataOnly Optimization", Once, - catalog.metadataOnlyOptimizer) - ) - - override def batches: Seq[Batch] = extendedHiveOptimizerRules ++ super.batches - } - /** * Planner that takes into account Hive-specific strategies. */ 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 5f896969188d..4f8b1cc21e09 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 @@ -183,6 +183,17 @@ private[hive] trait HiveClient { table: CatalogTable, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] + /** + * Returns the partitions for the given table that match the supplied partition spec. + * If no partition spec is specified, all partitions are returned. + */ + final def getPartitionsByFilter( + db: String, + table: String, + predicates: Seq[Expression]): Seq[CatalogTablePartition] = { + getPartitionsByFilter(getTable(db, table), predicates) + } + /** Returns partitions filtered by predicates for the given table. */ def getPartitionsByFilter( table: CatalogTable, 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 37b312da7763..503a5f1a8ee9 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 @@ -1687,8 +1687,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("spark-15752 metadata only optimizer") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - val df = Seq((1, 2), (3, 4)).toDF("key", "value") - df.createOrReplaceTempView("data") + val df = Seq((1, "2"), (3, "4")).toDF("key", "value") + df.createOrReplaceTempView("data_15752") sql( """ |CREATE TABLE srcpart_15752 (key INT, value STRING) @@ -1698,7 +1698,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql( s""" |INSERT OVERWRITE TABLE srcpart_15752 PARTITION (ds='$ds',hr='$hr') - |select key, value from data + |select key, value from data_15752 """.stripMargin) } checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) @@ -1716,7 +1716,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { sql( s""" |INSERT OVERWRITE TABLE srctext_15752 PARTITION (ds='$ds',hr='$hr') - |select key, value from data + |select key, value from data_15752 """.stripMargin) } From 1bb5812a3af9fffcbc879fba42ccfe66485617bb Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Sat, 25 Jun 2016 00:28:59 +0800 Subject: [PATCH 10/45] refactor --- .../catalyst/catalog/ExternalCatalog.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 2 +- .../sql/execution/MetadataOnlyOptimizer.scala | 162 ++++++++++++++++++ .../spark/sql/execution/SparkOptimizer.scala | 149 +--------------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 1 - 5 files changed, 169 insertions(+), 147 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala 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 5d478a4ccce9..005457b2bfbd 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 @@ -153,7 +153,7 @@ abstract class ExternalCatalog { def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition /** - * Returns partitions filtered by predicates for the given table, It just work for Hive. + * Returns partitions filtered by predicates for the given table, Filters only work for Hive. * * The filters Expressions may optionally be provided to filter the partitions returned. * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'), 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 be984bade600..c55830dfd798 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 @@ -586,7 +586,7 @@ class SessionCatalog( } /** - * Returns partitions filtered by predicates for the given table, It just work for Hive. + * Returns partitions filtered by predicates for the given table, Filters only work for Hive. * * The filters Expressions may optionally be provided to filter the partitions returned. * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala new file mode 100644 index 000000000000..6f4eb0e9c854 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, SessionCatalog} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} + +/** + * When scanning only partition columns, get results based on metadata without scanning files. + * It is used for distinct, distinct aggregations or distinct-like aggregations(example: Max/Min). + * Example: select Max(partition) from table. + */ +case class MetadataOnlyOptimizer(catalog: SessionCatalog) extends Rule[LogicalPlan] { + + private val sparkSession: SparkSession = SparkSession.getActiveSession.get + private def canSupportMetadataOnly(a: Aggregate): Boolean = { + val aggregateExpressions = a.aggregateExpressions.flatMap { expr => + expr.collect { + case agg: AggregateExpression => agg + } + }.distinct + if (aggregateExpressions.isEmpty) { + // Cannot support for aggregate that has no aggregateFunction. + // example: select col1 from table group by col1. + false + } else { + aggregateExpressions.forall { agg => + if (agg.isDistinct) { + true + } else { + // If function can be evaluated on just the distinct values of a column, it can be used + // by metadata-only optimizer. + agg.aggregateFunction match { + case max: Max => true + case min: Min => true + case hyperLog: HyperLogLogPlusPlus => true + case _ => false + } + } + } + } + } + + private def findRelation(plan: LogicalPlan): (Option[LogicalPlan], Seq[Expression]) = { + plan match { + case relation @ LogicalRelation(files: HadoopFsRelation, _, table) + if files.partitionSchema.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => + (Some(relation), Seq.empty[Expression]) + + case p @ Project(_, child) => + findRelation(child) + + case f @ Filter(filterCondition, child) => + val (plan, conditions) = findRelation(child) + (plan, conditions ++ Seq(filterCondition)) + + case _ => (None, Seq.empty[Expression]) + } + } + + private def convertToMetadataOnlyPlan( + parent: LogicalPlan, + project: Option[LogicalPlan], + filters: Seq[Expression], + relation: LogicalPlan): LogicalPlan = relation match { + case l @ LogicalRelation(files: HadoopFsRelation, _, _) => + val attributeMap = l.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = files.partitionSchema.map { field => + attributeMap.getOrElse(field.name, throw new AnalysisException( + s"Unable to resolve ${field.name} given [${l.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val selectedPartitions = files.location.listFiles(filters) + val partitionValues = selectedPartitions.map(_.values) + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + val scanPlan = project.map(_.withNewChildren(valuesPlan :: Nil)).getOrElse(valuesPlan) + parent.withNewChildren(scanPlan :: Nil) + } else { + parent + } + + case relation: CatalogRelation => + val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => + attributeMap.getOrElse(column, throw new AnalysisException( + s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val partitionColumnDataTypes = partitionColumns.map(_.dataType) + val partitionValues = catalog.getPartitionsByFilter(relation.catalogTable, filters) + .map { p => + InternalRow.fromSeq( + partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { + case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) + }) + } + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + val filterPlan = + filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) + val scanPlan = project.map(_.withNewChildren(filterPlan :: Nil)).getOrElse(filterPlan) + parent.withNewChildren(scanPlan :: Nil) + } else { + parent + } + + case _ => + parent + } + + def apply(plan: LogicalPlan): LogicalPlan = { + if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { + return plan + } + plan.transform { + case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => + val (plan, filters) = findRelation(child) + if (plan.isDefined) { + convertToMetadataOnlyPlan(a, None, filters, plan.get) + } else { + a + } + + case d @ Distinct(p @ Project(_, _)) => + val (plan, filters) = findRelation(p) + if (plan.isDefined) { + convertToMetadataOnlyPlan(d, Some(p), filters, plan.get) + } else { + d + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index e4ad530c4d3f..0512c7f72958 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -17,16 +17,12 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{AnalysisException, ExperimentalMethods, SparkSession} -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.optimizer._ -import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.ExperimentalMethods +import org.apache.spark.sql.catalyst.analysis.RewriteDistinctAggregates +import org.apache.spark.sql.catalyst.catalog.SessionCatalog +import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate import org.apache.spark.sql.internal.SQLConf @@ -44,138 +40,3 @@ class SparkOptimizer( Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } - -/** - * When scanning only partition columns, get results based on metadata without scanning files. - * It is used for distinct, distinct aggregations or distinct-like aggregations(example: Max/Min). - * Example: select Max(partition) from table. - */ -case class MetadataOnlyOptimizer(catalog: SessionCatalog) extends Rule[LogicalPlan] { - - private val sparkSession: SparkSession = SparkSession.getActiveSession.get - private def canSupportMetadataOnly(a: Aggregate): Boolean = { - val aggregateExpressions = a.aggregateExpressions.flatMap { expr => - expr.collect { - case agg: AggregateExpression => agg - } - }.distinct - if (aggregateExpressions.isEmpty) { - // Cannot support for aggregate that has no aggregateFunction. - // example: select col1 from table group by col1. - false - } else { - aggregateExpressions.forall { agg => - if (agg.isDistinct) { - true - } else { - // If function can be evaluated on just the distinct values of a column, it can be used - // by metadata-only optimizer. - agg.aggregateFunction match { - case max: Max => true - case min: Min => true - case hyperLog: HyperLogLogPlusPlus => true - case _ => false - } - } - } - } - } - - private def findRelation(plan: LogicalPlan): (Option[LogicalPlan], Seq[Expression]) = { - plan match { - case relation @ LogicalRelation(files: HadoopFsRelation, _, table) - if files.partitionSchema.nonEmpty => - (Some(relation), Seq.empty[Expression]) - - case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => - (Some(relation), Seq.empty[Expression]) - - case p @ Project(_, child) => - findRelation(child) - - case f @ Filter(filterCondition, child) => - val (plan, conditions) = findRelation(child) - (plan, conditions ++ Seq(filterCondition)) - - case _ => (None, Seq.empty[Expression]) - } - } - - private def convertToMetadataOnlyPlan( - parent: LogicalPlan, - project: Option[LogicalPlan], - filters: Seq[Expression], - relation: LogicalPlan): LogicalPlan = relation match { - case l @ LogicalRelation(files: HadoopFsRelation, _, _) => - val attributeMap = l.output.map(attr => (attr.name, attr)).toMap - val partitionColumns = files.partitionSchema.map { field => - attributeMap.getOrElse(field.name, throw new AnalysisException( - s"Unable to resolve ${field.name} given [${l.output.map(_.name).mkString(", ")}]")) - } - val filterColumns = filters.flatMap(_.references) - val projectSet = parent.references ++ AttributeSet(filterColumns) - if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val selectedPartitions = files.location.listFiles(filters) - val partitionValues = selectedPartitions.map(_.values) - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - val scanPlan = project.map(_.withNewChildren(valuesPlan :: Nil)).getOrElse(valuesPlan) - parent.withNewChildren(scanPlan :: Nil) - } else { - parent - } - - case relation: CatalogRelation => - val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap - val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => - attributeMap.getOrElse(column, throw new AnalysisException( - s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) - } - val filterColumns = filters.flatMap(_.references) - val projectSet = parent.references ++ AttributeSet(filterColumns) - if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val partitionColumnDataTypes = partitionColumns.map(_.dataType) - val partitionValues = catalog.getPartitionsByFilter(relation.catalogTable, filters) - .map { p => - InternalRow.fromSeq( - partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { - case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) - }) - } - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - val filterPlan = - filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) - val scanPlan = project.map(_.withNewChildren(filterPlan :: Nil)).getOrElse(filterPlan) - parent.withNewChildren(scanPlan :: Nil) - } else { - parent - } - - case _ => - parent - } - - def apply(plan: LogicalPlan): LogicalPlan = { - if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { - return plan - } - plan.transform { - case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => - val (plan, filters) = findRelation(child) - if (plan.isDefined) { - convertToMetadataOnlyPlan(a, None, filters, plan.get) - } else { - a - } - - case d @ Distinct(p @ Project(_, _)) => - val (plan, filters) = findRelation(p) - if (plan.isDefined) { - convertToMetadataOnlyPlan(d, Some(p), filters, plan.get) - } else { - d - } - } - } -} 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 3c7b70a4bd53..552eacfcbceb 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 @@ -455,7 +455,6 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log allowExisting) } } - } /** From 7e3729e407417b58bd5a406e43ab2b1901761699 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Sat, 25 Jun 2016 01:50:05 +0800 Subject: [PATCH 11/45] add ut --- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +++++++++++ .../spark/sql/hive/execution/SQLQuerySuite.scala | 2 +- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index bbe821b76805..68d922b454db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2870,4 +2870,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"SELECT '$literal' AS DUMMY"), Row(s"$expected") :: Nil) } + + test("spark-15752 metadata only optimizer for datasource table") { + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) + .toDF("id", "data", "partId", "part") + data.write.partitionBy("partId", "part").mode("append").saveAsTable("srcpart_15752") + checkAnswer(sql("select max(part) from srcpart_15752"), Row("odd")) + checkAnswer(sql("select max(part) from srcpart_15752 where partId = 0"), Row("even")) + checkAnswer(sql("select max(part) from (select part from srcpart_15752) t"), Row("odd")) + checkAnswer(sql("select distinct part from srcpart_15752"), Row("even") :: Row("odd") :: Nil) + checkAnswer(sql("select distinct part from srcpart_15752 where partId = 0"), Row("even")) + } } 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 503a5f1a8ee9..257acd067334 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 @@ -1685,7 +1685,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("spark-15752 metadata only optimizer") { + test("spark-15752 metadata only optimizer for hive table") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { val df = Seq((1, "2"), (3, "4")).toDF("key", "value") df.createOrReplaceTempView("data_15752") From fbf5d615713caab8a6d5aa8093cdb2ef59020966 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Sat, 25 Jun 2016 02:36:01 +0800 Subject: [PATCH 12/45] fix ut --- .../apache/spark/sql/execution/MetadataOnlyOptimizer.scala | 5 +++-- .../org/apache/spark/sql/execution/SparkOptimizer.scala | 5 +++-- .../scala/org/apache/spark/sql/internal/SessionState.scala | 3 ++- .../scala/org/apache/spark/sql/hive/HiveSessionState.scala | 3 +-- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 6f4eb0e9c854..f1dedec2051c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -31,9 +31,10 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela * It is used for distinct, distinct aggregations or distinct-like aggregations(example: Max/Min). * Example: select Max(partition) from table. */ -case class MetadataOnlyOptimizer(catalog: SessionCatalog) extends Rule[LogicalPlan] { +case class MetadataOnlyOptimizer( + sparkSession: SparkSession, + catalog: SessionCatalog) extends Rule[LogicalPlan] { - private val sparkSession: SparkSession = SparkSession.getActiveSession.get private def canSupportMetadataOnly(a: Aggregate): Boolean = { val aggregateExpressions = a.aggregateExpressions.flatMap { expr => expr.collect { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 0512c7f72958..49613e8b53d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.ExperimentalMethods +import org.apache.spark.sql.{ExperimentalMethods, SparkSession} import org.apache.spark.sql.catalyst.analysis.RewriteDistinctAggregates import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer @@ -27,13 +27,14 @@ import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate import org.apache.spark.sql.internal.SQLConf class SparkOptimizer( + sparkSession: SparkSession, catalog: SessionCatalog, conf: SQLConf, experimentalMethods: ExperimentalMethods) extends Optimizer(catalog, conf) { override val aggregateOptimizationsRules: Seq[Rule[LogicalPlan]] = - MetadataOnlyOptimizer(catalog) :: + MetadataOnlyOptimizer(sparkSession, catalog) :: RewriteDistinctAggregates :: Nil override def batches: Seq[Batch] = super.batches :+ 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 5f5cf5c6d30c..f10723618380 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 @@ -123,7 +123,8 @@ private[sql] class SessionState(sparkSession: SparkSession) { /** * Logical query plan optimizer. */ - lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods) + lazy val optimizer: Optimizer = + new SparkOptimizer(sparkSession, catalog, conf, experimentalMethods) /** * Parser that extracts expressions, plans, table identifiers etc. from SQL texts. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 46978d5e64a2..8773993d362c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Analyzer -import org.apache.spark.sql.catalyst.optimizer.Optimizer -import org.apache.spark.sql.execution.{SparkOptimizer, SparkPlanner} +import org.apache.spark.sql.execution.SparkPlanner import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.SessionState From 3411fd6e51758d8f556b9cbf3abbc1a7fda952e1 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Mon, 27 Jun 2016 01:29:19 +0800 Subject: [PATCH 13/45] fix project --- .../sql/catalyst/optimizer/Optimizer.scala | 3 +- .../sql/execution/MetadataOnlyOptimizer.scala | 63 +++++++++++++------ .../sql/hive/execution/SQLQuerySuite.scala | 6 ++ 3 files changed, 52 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3b567571a07c..1b70aa426691 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -53,8 +53,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) EliminateSubqueryAliases, ReplaceExpressions, ComputeCurrentTime, - GetCurrentDatabase(sessionCatalog), - RewriteDistinctAggregates) :: + GetCurrentDatabase(sessionCatalog)) :: ////////////////////////////////////////////////////////////////////////////////////////// // Optimizer rules start here ////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index f1dedec2051c..1ffb4e135da4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -63,29 +63,50 @@ case class MetadataOnlyOptimizer( } } - private def findRelation(plan: LogicalPlan): (Option[LogicalPlan], Seq[Expression]) = { + private def collectAliases(fields: Seq[Expression]): Map[ExprId, Expression] = fields.collect { + case a @ Alias(child, _) => a.toAttribute.exprId -> child + }.toMap + + private def substitute(aliases: Map[ExprId, Expression])(expr: Expression): Expression = { + expr.transform { + case a @ Alias(ref: AttributeReference, name) => + aliases.get(ref.exprId) + .map(Alias(_, name)(a.exprId, a.qualifier, isGenerated = a.isGenerated)) + .getOrElse(a) + + case a: AttributeReference => + aliases.get(a.exprId) + .map(Alias(_, a.name)(a.exprId, a.qualifier, isGenerated = a.isGenerated)).getOrElse(a) + } + } + + private def findRelation(plan: LogicalPlan) + : (Option[LogicalPlan], Seq[NamedExpression], Seq[Expression], Map[ExprId, Expression]) = { plan match { case relation @ LogicalRelation(files: HadoopFsRelation, _, table) if files.partitionSchema.nonEmpty => - (Some(relation), Seq.empty[Expression]) + (Some(relation), Seq.empty[NamedExpression], Seq.empty[Expression], Map.empty) case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => - (Some(relation), Seq.empty[Expression]) + (Some(relation), Seq.empty[NamedExpression], Seq.empty[Expression], Map.empty) - case p @ Project(_, child) => - findRelation(child) + case p @ Project(fields, child) if fields.forall(_.deterministic) => + val (plan, _, filters, aliases) = findRelation(child) + val substitutedFields = fields.map(substitute(aliases)).asInstanceOf[Seq[NamedExpression]] + (plan, substitutedFields, filters, collectAliases(substitutedFields)) - case f @ Filter(filterCondition, child) => - val (plan, conditions) = findRelation(child) - (plan, conditions ++ Seq(filterCondition)) + case f @ Filter(condition, child) if condition.deterministic => + val (plan, fields, filters, aliases) = findRelation(child) + val substitutedCondition = substitute(aliases)(condition) + (plan, fields, filters ++ Seq(substitutedCondition), aliases) - case _ => (None, Seq.empty[Expression]) + case _ => (None, Seq.empty[NamedExpression], Seq.empty[Expression], Map.empty) } } private def convertToMetadataOnlyPlan( parent: LogicalPlan, - project: Option[LogicalPlan], + projectList: Seq[NamedExpression], filters: Seq[Expression], relation: LogicalPlan): LogicalPlan = relation match { case l @ LogicalRelation(files: HadoopFsRelation, _, _) => @@ -101,8 +122,11 @@ case class MetadataOnlyOptimizer( val partitionValues = selectedPartitions.map(_.values) val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - val scanPlan = project.map(_.withNewChildren(valuesPlan :: Nil)).getOrElse(valuesPlan) - parent.withNewChildren(scanPlan :: Nil) + if (projectList.nonEmpty) { + parent.withNewChildren(Project(projectList, valuesPlan) :: Nil) + } else { + parent.withNewChildren(valuesPlan :: Nil) + } } else { parent } @@ -128,8 +152,11 @@ case class MetadataOnlyOptimizer( val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) val filterPlan = filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) - val scanPlan = project.map(_.withNewChildren(filterPlan :: Nil)).getOrElse(filterPlan) - parent.withNewChildren(scanPlan :: Nil) + if (projectList.nonEmpty) { + parent.withNewChildren(Project(projectList, filterPlan) :: Nil) + } else { + parent.withNewChildren(filterPlan :: Nil) + } } else { parent } @@ -144,17 +171,17 @@ case class MetadataOnlyOptimizer( } plan.transform { case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => - val (plan, filters) = findRelation(child) + val (plan, projectList, filters, _) = findRelation(child) if (plan.isDefined) { - convertToMetadataOnlyPlan(a, None, filters, plan.get) + convertToMetadataOnlyPlan(a, projectList, filters, plan.get) } else { a } case d @ Distinct(p @ Project(_, _)) => - val (plan, filters) = findRelation(p) + val (plan, projectList, filters, _) = findRelation(p) if (plan.isDefined) { - convertToMetadataOnlyPlan(d, Some(p), filters, plan.get) + convertToMetadataOnlyPlan(d, projectList, filters, plan.get) } else { d } 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 257acd067334..e4608d4bb740 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 @@ -1723,8 +1723,14 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) + checkAnswer( + sql("select max(x) from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + Row(13)) checkAnswer(sql("select distinct hr from srctext_15752"), Row(11) :: Row(12) :: Nil) checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) + checkAnswer( + sql("select distinct x from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + Row(13)) } } } From aefab7fa2b2c99f861faf0693fc1e13c2cf5311c Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Mon, 27 Jun 2016 21:47:29 +0800 Subject: [PATCH 14/45] address comments --- .../sql/execution/MetadataOnlyOptimizer.scala | 25 ++++++++----------- .../sql/hive/execution/SQLQuerySuite.scala | 2 ++ 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 1ffb4e135da4..7d854d7bc476 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -42,9 +42,9 @@ case class MetadataOnlyOptimizer( } }.distinct if (aggregateExpressions.isEmpty) { - // Cannot support for aggregate that has no aggregateFunction. - // example: select col1 from table group by col1. - false + // Support for aggregate that has no aggregateFunction when expressions are partition columns + // example: select partitionCol from table group by partitionCol. + true } else { aggregateExpressions.forall { agg => if (agg.isDistinct) { @@ -119,13 +119,11 @@ case class MetadataOnlyOptimizer( val projectSet = parent.references ++ AttributeSet(filterColumns) if (projectSet.subsetOf(AttributeSet(partitionColumns))) { val selectedPartitions = files.location.listFiles(filters) - val partitionValues = selectedPartitions.map(_.values) - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesRdd = sparkSession.sparkContext.parallelize(selectedPartitions.map(_.values), 1) val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - if (projectList.nonEmpty) { - parent.withNewChildren(Project(projectList, valuesPlan) :: Nil) - } else { - parent.withNewChildren(valuesPlan :: Nil) + parent.transform { + case l @ LogicalRelation(files: HadoopFsRelation, _, _) => + valuesPlan } } else { parent @@ -150,12 +148,9 @@ case class MetadataOnlyOptimizer( } val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - val filterPlan = - filters.reduceLeftOption(And).map(Filter(_, valuesPlan)).getOrElse(valuesPlan) - if (projectList.nonEmpty) { - parent.withNewChildren(Project(projectList, filterPlan) :: Nil) - } else { - parent.withNewChildren(filterPlan :: Nil) + parent.transform { + case relation: CatalogRelation => + valuesPlan } } else { parent 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 e4608d4bb740..b408d42fafd2 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 @@ -1701,6 +1701,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |select key, value from data_15752 """.stripMargin) } + checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) @@ -1720,6 +1721,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) } + checkAnswer(sql("select hr from srctext_15752 where hr = 11 group by hr"), Row(11)) checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) From c5ccdea435789ec5cdea1438ace09d91e7726d22 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 28 Jun 2016 00:23:20 +0800 Subject: [PATCH 15/45] fix cube/rollup --- .../catalyst/catalog/ExternalCatalog.scala | 13 ----------- .../catalyst/catalog/InMemoryCatalog.scala | 7 ------ .../sql/catalyst/catalog/SessionCatalog.scala | 13 ----------- .../sql/catalyst/optimizer/Optimizer.scala | 7 ++---- .../sql/execution/MetadataOnlyOptimizer.scala | 23 +++++++++++-------- .../spark/sql/execution/SparkOptimizer.scala | 5 +--- .../spark/sql/hive/HiveExternalCatalog.scala | 9 -------- .../spark/sql/hive/client/HiveClient.scala | 11 --------- .../sql/hive/execution/SQLQuerySuite.scala | 6 +++++ 9 files changed, 22 insertions(+), 72 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 005457b2bfbd..2745d5b0dd3b 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 @@ -152,19 +152,6 @@ abstract class ExternalCatalog { def getPartition(db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition - /** - * Returns partitions filtered by predicates for the given table, Filters only work for Hive. - * - * The filters Expressions may optionally be provided to filter the partitions returned. - * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'), - * then the filters (a='1') will return the first two only. - * @param catalogTable table - * @param filters The filters used to prune which partitions are returned. - */ - def getPartitionsByFilter( - catalogTable: CatalogTable, - filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] - /** * List the metadata of all partitions that belong to the specified table, assuming it exists. * 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 1d9b30d3f873..097ab99c6452 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 @@ -448,13 +448,6 @@ class InMemoryCatalog(hadoopConfig: Configuration = new Configuration) extends E catalog(db).tables(table).partitions(spec) } - override def getPartitionsByFilter( - catalogTable: CatalogTable, - filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] = synchronized { - requireTableExists(catalogTable.database, catalogTable.identifier.table) - catalog(catalogTable.database).tables(catalogTable.identifier.table).partitions.values.toSeq - } - override def listPartitions( db: String, table: String, 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 c55830dfd798..7ab10d1c3823 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 @@ -585,19 +585,6 @@ class SessionCatalog( externalCatalog.getPartition(db, table, spec) } - /** - * Returns partitions filtered by predicates for the given table, Filters only work for Hive. - * - * The filters Expressions may optionally be provided to filter the partitions returned. - * For instance, if there exist partitions (a='1', b='2'), (a='1', b='3') and (a='2', b='4'), - * then the filters (a='1') will return the first two only. - */ - def getPartitionsByFilter( - catalogTable: CatalogTable, - filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] = { - externalCatalog.getPartitionsByFilter(catalogTable, filters) - } - /** * List the metadata of all partitions that belong to the specified table, assuming it exists. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 1b70aa426691..2bca31d5f1f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -42,8 +42,6 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) protected val fixedPoint = FixedPoint(conf.optimizerMaxIterations) - val aggregateOptimizationsRules: Seq[Rule[LogicalPlan]] = RewriteDistinctAggregates :: Nil - def batches: Seq[Batch] = { // Technically some of the rules in Finish Analysis are not optimizer rules and belong more // in the analyzer, because they are needed for correctness (e.g. ComputeCurrentTime). @@ -53,7 +51,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) EliminateSubqueryAliases, ReplaceExpressions, ComputeCurrentTime, - GetCurrentDatabase(sessionCatalog)) :: + GetCurrentDatabase(sessionCatalog), + RewriteDistinctAggregates) :: ////////////////////////////////////////////////////////////////////////////////////////// // Optimizer rules start here ////////////////////////////////////////////////////////////////////////////////////////// @@ -62,8 +61,6 @@ abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf) // extra operators between two adjacent Union operators. // - Call CombineUnions again in Batch("Operator Optimizations"), // since the other rules might make two separate Unions operators adjacent. - Batch("Aggregate Optimizations", Once, - aggregateOptimizationsRules : _*) :: Batch("Union", Once, CombineUnions) :: Batch("Subquery", Once, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 7d854d7bc476..4b2e09dd3c90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -29,7 +29,14 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela /** * When scanning only partition columns, get results based on metadata without scanning files. * It is used for distinct, distinct aggregations or distinct-like aggregations(example: Max/Min). - * Example: select Max(partition) from table. + * First of all, scanning only partition columns are required, then the rule does the following + * things here: + * 1. aggregate expression is partition columns, + * e.g. SELECT col FROM tbl GROUP BY col or SELECT col FROM tbl GROUP BY cube(col). + * 2. aggregate function on partition columns with DISTINCT, + * e.g. SELECT count(DISTINCT col) FROM tbl GROUP BY col. + * 3. aggregate function on partition columns which have same result with DISTINCT keyword. + * e.g. SELECT Max(col2) FROM tbl GROUP BY col1. */ case class MetadataOnlyOptimizer( sparkSession: SparkSession, @@ -44,6 +51,7 @@ case class MetadataOnlyOptimizer( if (aggregateExpressions.isEmpty) { // Support for aggregate that has no aggregateFunction when expressions are partition columns // example: select partitionCol from table group by partitionCol. + // Moreover, multiple-distinct has been rewritted into it by RewriteDistinctAggregates. true } else { aggregateExpressions.forall { agg => @@ -100,6 +108,9 @@ case class MetadataOnlyOptimizer( val substitutedCondition = substitute(aliases)(condition) (plan, fields, filters ++ Seq(substitutedCondition), aliases) + case e @ Expand(_, _, child) => + findRelation(child) + case _ => (None, Seq.empty[NamedExpression], Seq.empty[Expression], Map.empty) } } @@ -139,7 +150,7 @@ case class MetadataOnlyOptimizer( val projectSet = parent.references ++ AttributeSet(filterColumns) if (projectSet.subsetOf(AttributeSet(partitionColumns))) { val partitionColumnDataTypes = partitionColumns.map(_.dataType) - val partitionValues = catalog.getPartitionsByFilter(relation.catalogTable, filters) + val partitionValues = catalog.listPartitions(relation.catalogTable.identifier) .map { p => InternalRow.fromSeq( partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { @@ -172,14 +183,6 @@ case class MetadataOnlyOptimizer( } else { a } - - case d @ Distinct(p @ Project(_, _)) => - val (plan, projectList, filters, _) = findRelation(p) - if (plan.isDefined) { - convertToMetadataOnlyPlan(d, projectList, filters, plan.get) - } else { - d - } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 49613e8b53d5..dd49bf989e77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -33,11 +33,8 @@ class SparkOptimizer( experimentalMethods: ExperimentalMethods) extends Optimizer(catalog, conf) { - override val aggregateOptimizationsRules: Seq[Rule[LogicalPlan]] = - MetadataOnlyOptimizer(sparkSession, catalog) :: - RewriteDistinctAggregates :: Nil - override def batches: Seq[Batch] = super.batches :+ + Batch("Metadata Only Optimization", Once, MetadataOnlyOptimizer(sparkSession, catalog)) :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } 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 bfb53b915da8..001dd07c8763 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 @@ -323,15 +323,6 @@ private[spark] class HiveExternalCatalog(client: HiveClient, hadoopConf: Configu client.getPartition(db, table, spec) } - /** - * Returns partitions filtered by predicates for the given table. - */ - override def getPartitionsByFilter( - catalogTable: CatalogTable, - filters: Seq[Expression] = Nil): Seq[CatalogTablePartition] = withClient { - client.getPartitionsByFilter(catalogTable, filters) - } - /** * Returns the partition names from hive metastore for a given table in a database. */ 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 4f8b1cc21e09..5f896969188d 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 @@ -183,17 +183,6 @@ private[hive] trait HiveClient { table: CatalogTable, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] - /** - * Returns the partitions for the given table that match the supplied partition spec. - * If no partition spec is specified, all partitions are returned. - */ - final def getPartitionsByFilter( - db: String, - table: String, - predicates: Seq[Expression]): Seq[CatalogTablePartition] = { - getPartitionsByFilter(getTable(db, table), predicates) - } - /** Returns partitions filtered by predicates for the given table. */ def getPartitionsByFilter( table: CatalogTable, 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 b408d42fafd2..939c53ec2223 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 @@ -1702,6 +1702,9 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) } checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) + checkAnswer( + sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), + Row(null) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) @@ -1722,6 +1725,9 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } checkAnswer(sql("select hr from srctext_15752 where hr = 11 group by hr"), Row(11)) + checkAnswer( + sql("select hr from srctext_15752 where hr = 12 group by rollup(hr)"), + Row(null) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) From ae6cf9ff909e4c375d4d0f688603f415edff6dca Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 28 Jun 2016 00:44:48 +0800 Subject: [PATCH 16/45] fix style --- .../apache/spark/sql/catalyst/catalog/ExternalCatalog.scala | 1 - .../apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala | 1 - .../scala/org/apache/spark/sql/execution/SparkOptimizer.scala | 3 --- .../scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala | 1 - 4 files changed, 6 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 2745d5b0dd3b..6714846e8cbd 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 @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException -import org.apache.spark.sql.catalyst.expressions.Expression /** 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 097ab99c6452..fb3e1b3637f2 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 @@ -28,7 +28,6 @@ import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index dd49bf989e77..9a680bdac962 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -18,11 +18,8 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{ExperimentalMethods, SparkSession} -import org.apache.spark.sql.catalyst.analysis.RewriteDistinctAggregates import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate import org.apache.spark.sql.internal.SQLConf 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 001dd07c8763..b8bc9ab900ad 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 @@ -30,7 +30,6 @@ 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.expressions.Expression import org.apache.spark.sql.hive.client.HiveClient From 159331b4cc0583b6414152512d84696c16b895fc Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 28 Jun 2016 02:32:13 +0800 Subject: [PATCH 17/45] refactor --- .../sql/execution/MetadataOnlyOptimizer.scala | 105 ++++++++++-------- 1 file changed, 56 insertions(+), 49 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 4b2e09dd3c90..3162d5f59208 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -115,60 +115,60 @@ case class MetadataOnlyOptimizer( } } - private def convertToMetadataOnlyPlan( + private def convertLogicalToMetadataOnly( parent: LogicalPlan, - projectList: Seq[NamedExpression], filters: Seq[Expression], - relation: LogicalPlan): LogicalPlan = relation match { - case l @ LogicalRelation(files: HadoopFsRelation, _, _) => - val attributeMap = l.output.map(attr => (attr.name, attr)).toMap - val partitionColumns = files.partitionSchema.map { field => - attributeMap.getOrElse(field.name, throw new AnalysisException( - s"Unable to resolve ${field.name} given [${l.output.map(_.name).mkString(", ")}]")) - } - val filterColumns = filters.flatMap(_.references) - val projectSet = parent.references ++ AttributeSet(filterColumns) - if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val selectedPartitions = files.location.listFiles(filters) - val valuesRdd = sparkSession.sparkContext.parallelize(selectedPartitions.map(_.values), 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - parent.transform { - case l @ LogicalRelation(files: HadoopFsRelation, _, _) => - valuesPlan - } - } else { - parent + logical: LogicalPlan, + files: HadoopFsRelation): LogicalPlan = { + val attributeMap = logical.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = files.partitionSchema.map { field => + attributeMap.getOrElse(field.name, throw new AnalysisException( + s"Unable to resolve ${field.name} given [${logical.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val selectedPartitions = files.location.listFiles(filters) + val valuesRdd = sparkSession.sparkContext.parallelize(selectedPartitions.map(_.values), 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + parent.transform { + case l @ LogicalRelation(files: HadoopFsRelation, _, _) => + valuesPlan } + } else { + parent + } + } - case relation: CatalogRelation => - val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap - val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => - attributeMap.getOrElse(column, throw new AnalysisException( - s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) - } - val filterColumns = filters.flatMap(_.references) - val projectSet = parent.references ++ AttributeSet(filterColumns) - if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val partitionColumnDataTypes = partitionColumns.map(_.dataType) - val partitionValues = catalog.listPartitions(relation.catalogTable.identifier) - .map { p => - InternalRow.fromSeq( - partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { - case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) - }) - } - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - parent.transform { - case relation: CatalogRelation => - valuesPlan + private def convertCatalogToMetadataOnly( + parent: LogicalPlan, + filters: Seq[Expression], + relation: CatalogRelation): LogicalPlan = { + val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap + val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => + attributeMap.getOrElse(column, throw new AnalysisException( + s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) + } + val filterColumns = filters.flatMap(_.references) + val projectSet = parent.references ++ AttributeSet(filterColumns) + if (projectSet.subsetOf(AttributeSet(partitionColumns))) { + val partitionColumnDataTypes = partitionColumns.map(_.dataType) + val partitionValues = catalog.listPartitions(relation.catalogTable.identifier) + .map { p => + InternalRow.fromSeq( + partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { + case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) + }) } - } else { - parent + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + parent.transform { + case relation: CatalogRelation => + valuesPlan } - - case _ => + } else { parent + } } def apply(plan: LogicalPlan): LogicalPlan = { @@ -177,9 +177,16 @@ case class MetadataOnlyOptimizer( } plan.transform { case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => - val (plan, projectList, filters, _) = findRelation(child) + val (plan, _, filters, _) = findRelation(child) if (plan.isDefined) { - convertToMetadataOnlyPlan(a, projectList, filters, plan.get) + plan.get match { + case l @ LogicalRelation(files: HadoopFsRelation, _, _) => + convertLogicalToMetadataOnly(a, filters, l, files) + case relation: CatalogRelation => + convertCatalogToMetadataOnly(a, filters, relation) + case _ => + a + } } else { a } From 3a1438ba41c94ed44dbc0dc43c2457509f5e4fcc Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 28 Jun 2016 17:41:36 +0800 Subject: [PATCH 18/45] refactor --- .../sql/catalyst/catalog/interface.scala | 2 +- .../sql/execution/MetadataOnlyOptimizer.scala | 130 +++++++----------- .../sql/hive/execution/SQLQuerySuite.scala | 9 ++ 3 files changed, 63 insertions(+), 78 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 6197acab3378..43c5e7b2992b 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 @@ -231,7 +231,7 @@ object CatalogTypes { * If we can in the future consolidate SimpleCatalogRelation and MetastoreRelation, we should * probably remove this interface. */ -trait CatalogRelation { +trait CatalogRelation extends LogicalPlan { def catalogTable: CatalogTable def output: Seq[Attribute] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 3162d5f59208..ba1cfcdc3205 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -71,86 +71,37 @@ case class MetadataOnlyOptimizer( } } - private def collectAliases(fields: Seq[Expression]): Map[ExprId, Expression] = fields.collect { - case a @ Alias(child, _) => a.toAttribute.exprId -> child - }.toMap - - private def substitute(aliases: Map[ExprId, Expression])(expr: Expression): Expression = { - expr.transform { - case a @ Alias(ref: AttributeReference, name) => - aliases.get(ref.exprId) - .map(Alias(_, name)(a.exprId, a.qualifier, isGenerated = a.isGenerated)) - .getOrElse(a) - - case a: AttributeReference => - aliases.get(a.exprId) - .map(Alias(_, a.name)(a.exprId, a.qualifier, isGenerated = a.isGenerated)).getOrElse(a) - } - } - - private def findRelation(plan: LogicalPlan) - : (Option[LogicalPlan], Seq[NamedExpression], Seq[Expression], Map[ExprId, Expression]) = { - plan match { - case relation @ LogicalRelation(files: HadoopFsRelation, _, table) - if files.partitionSchema.nonEmpty => - (Some(relation), Seq.empty[NamedExpression], Seq.empty[Expression], Map.empty) - - case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => - (Some(relation), Seq.empty[NamedExpression], Seq.empty[Expression], Map.empty) - - case p @ Project(fields, child) if fields.forall(_.deterministic) => - val (plan, _, filters, aliases) = findRelation(child) - val substitutedFields = fields.map(substitute(aliases)).asInstanceOf[Seq[NamedExpression]] - (plan, substitutedFields, filters, collectAliases(substitutedFields)) - - case f @ Filter(condition, child) if condition.deterministic => - val (plan, fields, filters, aliases) = findRelation(child) - val substitutedCondition = substitute(aliases)(condition) - (plan, fields, filters ++ Seq(substitutedCondition), aliases) - - case e @ Expand(_, _, child) => - findRelation(child) - - case _ => (None, Seq.empty[NamedExpression], Seq.empty[Expression], Map.empty) - } - } - private def convertLogicalToMetadataOnly( - parent: LogicalPlan, - filters: Seq[Expression], - logical: LogicalPlan, + project: LogicalPlan, + filter: Option[Expression], + logical: LogicalRelation, files: HadoopFsRelation): LogicalPlan = { val attributeMap = logical.output.map(attr => (attr.name, attr)).toMap val partitionColumns = files.partitionSchema.map { field => attributeMap.getOrElse(field.name, throw new AnalysisException( s"Unable to resolve ${field.name} given [${logical.output.map(_.name).mkString(", ")}]")) } - val filterColumns = filters.flatMap(_.references) - val projectSet = parent.references ++ AttributeSet(filterColumns) + val projectSet = filter.map(project.references ++ _.references).getOrElse(project.references) if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val selectedPartitions = files.location.listFiles(filters) + val selectedPartitions = files.location.listFiles(filter.map(Seq(_)).getOrElse(Seq.empty)) val valuesRdd = sparkSession.sparkContext.parallelize(selectedPartitions.map(_.values), 1) val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - parent.transform { - case l @ LogicalRelation(files: HadoopFsRelation, _, _) => - valuesPlan - } + valuesPlan } else { - parent + logical } } private def convertCatalogToMetadataOnly( - parent: LogicalPlan, - filters: Seq[Expression], + project: LogicalPlan, + filter: Option[Expression], relation: CatalogRelation): LogicalPlan = { val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => attributeMap.getOrElse(column, throw new AnalysisException( s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) } - val filterColumns = filters.flatMap(_.references) - val projectSet = parent.references ++ AttributeSet(filterColumns) + val projectSet = filter.map(project.references ++ _.references).getOrElse(project.references) if (projectSet.subsetOf(AttributeSet(partitionColumns))) { val partitionColumnDataTypes = partitionColumns.map(_.dataType) val partitionValues = catalog.listPartitions(relation.catalogTable.identifier) @@ -162,34 +113,59 @@ case class MetadataOnlyOptimizer( } val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - parent.transform { - case relation: CatalogRelation => - valuesPlan - } + valuesPlan } else { - parent + relation } } + private def convertToMetadataOnly(plan: LogicalPlan): LogicalPlan = plan match { + case p @ Project(fields, child) => + child match { + case f @ Filter(condition, l @ LogicalRelation(files: HadoopFsRelation, _, _)) + if files.partitionSchema.nonEmpty => + val plan = convertLogicalToMetadataOnly(p, Some(condition), l, files) + p.withNewChildren(f.withNewChildren(plan :: Nil) :: Nil) + + case l @ LogicalRelation(files: HadoopFsRelation, _, _) + if files.partitionSchema.nonEmpty => + val plan = convertLogicalToMetadataOnly(p, None, l, files) + p.withNewChildren(plan :: Nil) + + case f @ Filter(condition, relation: CatalogRelation) + if relation.catalogTable.partitionColumnNames.nonEmpty => + val plan = convertCatalogToMetadataOnly(p, Some(condition), relation) + p.withNewChildren(f.withNewChildren(plan :: Nil) :: Nil) + + case relation: CatalogRelation + if relation.catalogTable.partitionColumnNames.nonEmpty => + val plan = convertCatalogToMetadataOnly(p, None, relation) + p.withNewChildren(plan :: Nil) + + case other => + p.withNewChildren(p.children.map(convertToMetadataOnly(_))) + } + + case f : Filter => + f.withNewChildren(f.children.map(convertToMetadataOnly(_))) + + case e: Expand => + e.withNewChildren(e.children.map(convertToMetadataOnly(_))) + + case u: Union => + u.withNewChildren(u.children.map(convertToMetadataOnly(_))) + + case other: LogicalPlan => + other + } + def apply(plan: LogicalPlan): LogicalPlan = { if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { return plan } plan.transform { case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => - val (plan, _, filters, _) = findRelation(child) - if (plan.isDefined) { - plan.get match { - case l @ LogicalRelation(files: HadoopFsRelation, _, _) => - convertLogicalToMetadataOnly(a, filters, l, files) - case relation: CatalogRelation => - convertCatalogToMetadataOnly(a, filters, relation) - case _ => - a - } - } else { - a - } + a.withNewChildren(convertToMetadataOnly(child) :: Nil) } } } 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 939c53ec2223..f17d0ce55d80 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 @@ -1701,10 +1701,15 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |select key, value from data_15752 """.stripMargin) } + checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) checkAnswer( sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), Row(null) :: Row(12) :: Nil) + checkAnswer( + sql("select hr from (select hr from srcpart_15752 where hr = 11 union all " + + "select hr from srcpart_15752 where hr= 12)t group by hr"), + Row(11) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) @@ -1728,6 +1733,10 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { checkAnswer( sql("select hr from srctext_15752 where hr = 12 group by rollup(hr)"), Row(null) :: Row(12) :: Nil) + checkAnswer( + sql("select hr from (select hr from srctext_15752 where hr = 11 union all " + + "select hr from srctext_15752 where hr= 12)t group by hr"), + Row(11) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) From c0a7d59b014f55b348aa5ed5b089ee87936d2413 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 29 Jun 2016 19:54:45 +0800 Subject: [PATCH 19/45] update --- .../sql/catalyst/analysis/Analyzer.scala | 5 +- .../expressions/namedExpressions.scala | 39 ++++-- .../sql/execution/MetadataOnlyOptimizer.scala | 123 ++++++++---------- .../datasources/LogicalRelation.scala | 5 +- .../datasources/fileSourceInterfaces.scala | 2 + .../apache/spark/sql/sources/interfaces.scala | 1 + .../spark/sql/hive/MetastoreRelation.scala | 8 +- .../sql/hive/execution/SQLQuerySuite.scala | 28 ++-- 8 files changed, 108 insertions(+), 103 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 96f2e38946f1..044121ea45a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -292,14 +292,15 @@ class Analyzer( // Ensure all the expressions have been resolved. case x: GroupingSets if x.expressions.forall(_.resolved) => - val gid = AttributeReference(VirtualColumn.groupingIdName, IntegerType, false)() + val gid = AttributeReference(VirtualColumn.groupingIdName, IntegerType, false)( + isPartitionColumn = true) // Expand works by setting grouping expressions to null as determined by the bitmasks. To // prevent these null values from being used in an aggregate instead of the original value // we need to create new aliases for all group by expressions that will only be used for // the intended purpose. val groupByAliases: Seq[Alias] = x.groupByExprs.map { - case e: NamedExpression => Alias(e, e.name)() + case e: NamedExpression => Alias(e, e.name)(isPartitionColumn = e.isPartitionColumn) case other => Alias(other, other.toString)() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 306a99d5a37b..ba7237dd1825 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -83,6 +83,9 @@ trait NamedExpression extends Expression { /** Returns true if the expression is generated by Catalyst */ def isGenerated: java.lang.Boolean = false + /** Returns true if the expression is a partition column */ + def isPartitionColumn: java.lang.Boolean = false + /** Returns a copy of this expression with a new `exprId`. */ def newInstance(): NamedExpression @@ -127,12 +130,14 @@ abstract class Attribute extends LeafExpression with NamedExpression with NullIn * tableName and subQueryAlias are possible qualifiers. * @param explicitMetadata Explicit metadata associated with this alias that overwrites child's. * @param isGenerated A flag to indicate if this alias is generated by Catalyst + * @param isPartitionColumn A flag to indicate if this alias is a partition column */ case class Alias(child: Expression, name: String)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, val explicitMetadata: Option[Metadata] = None, - override val isGenerated: java.lang.Boolean = false) + override val isGenerated: java.lang.Boolean = false, + override val isPartitionColumn: java.lang.Boolean = false) extends UnaryExpression with NamedExpression { // Alias(Generator, xx) need to be transformed into Generate(generator, ...) @@ -158,12 +163,13 @@ case class Alias(child: Expression, name: String)( def newInstance(): NamedExpression = Alias(child, name)( - qualifier = qualifier, explicitMetadata = explicitMetadata, isGenerated = isGenerated) + qualifier = qualifier, explicitMetadata = explicitMetadata, isGenerated = isGenerated, + isPartitionColumn = isPartitionColumn) override def toAttribute: Attribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable, metadata)( - exprId, qualifier, isGenerated) + exprId, qualifier, isGenerated, isPartitionColumn) } else { UnresolvedAttribute(name) } @@ -172,7 +178,7 @@ case class Alias(child: Expression, name: String)( override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: explicitMetadata :: isGenerated :: Nil + exprId :: qualifier :: explicitMetadata :: isGenerated :: isPartitionColumn :: Nil } override def hashCode(): Int = { @@ -206,6 +212,7 @@ case class Alias(child: Expression, name: String)( * qualified way. Consider the examples tableName.name, subQueryAlias.name. * tableName and subQueryAlias are possible qualifiers. * @param isGenerated A flag to indicate if this reference is generated by Catalyst + * @param isPartitionColumn A flag to indicate if this reference is a partition column */ case class AttributeReference( name: String, @@ -214,7 +221,8 @@ case class AttributeReference( override val metadata: Metadata = Metadata.empty)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, - override val isGenerated: java.lang.Boolean = false) + override val isGenerated: java.lang.Boolean = false, + override val isPartitionColumn: java.lang.Boolean = false) extends Attribute with Unevaluable { /** @@ -252,7 +260,7 @@ case class AttributeReference( override def newInstance(): AttributeReference = AttributeReference(name, dataType, nullable, metadata)( - qualifier = qualifier, isGenerated = isGenerated) + qualifier = qualifier, isGenerated = isGenerated, isPartitionColumn = isPartitionColumn) /** * Returns a copy of this [[AttributeReference]] with changed nullability. @@ -261,7 +269,8 @@ case class AttributeReference( if (nullable == newNullability) { this } else { - AttributeReference(name, dataType, newNullability, metadata)(exprId, qualifier, isGenerated) + AttributeReference(name, dataType, newNullability, metadata)( + exprId, qualifier, isGenerated, isPartitionColumn) } } @@ -269,7 +278,8 @@ case class AttributeReference( if (name == newName) { this } else { - AttributeReference(newName, dataType, nullable, metadata)(exprId, qualifier, isGenerated) + AttributeReference(newName, dataType, nullable, metadata)( + exprId, qualifier, isGenerated, isPartitionColumn) } } @@ -280,7 +290,8 @@ case class AttributeReference( if (newQualifier == qualifier) { this } else { - AttributeReference(name, dataType, nullable, metadata)(exprId, newQualifier, isGenerated) + AttributeReference(name, dataType, nullable, metadata)( + exprId, newQualifier, isGenerated, isPartitionColumn) } } @@ -288,12 +299,18 @@ case class AttributeReference( if (exprId == newExprId) { this } else { - AttributeReference(name, dataType, nullable, metadata)(newExprId, qualifier, isGenerated) + AttributeReference(name, dataType, nullable, metadata)( + newExprId, qualifier, isGenerated, isPartitionColumn) } } + def setPartitionColumn(isPartitionColumn: Boolean): AttributeReference = { + AttributeReference(name, dataType, nullable, metadata)( + exprId, qualifier, isGenerated, isPartitionColumn) + } + override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: isGenerated :: Nil + exprId :: qualifier :: isGenerated :: isPartitionColumn :: Nil } override def toString: String = s"$name#${exprId.id}$typeSuffix" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index ba1cfcdc3205..9709a9871d51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -28,11 +28,10 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela /** * When scanning only partition columns, get results based on metadata without scanning files. - * It is used for distinct, distinct aggregations or distinct-like aggregations(example: Max/Min). - * First of all, scanning only partition columns are required, then the rule does the following - * things here: + * It's used for operators that only need distinct values. Currently only [[Aggregate]] operator + * which satisfy the following conditions are supported: * 1. aggregate expression is partition columns, - * e.g. SELECT col FROM tbl GROUP BY col or SELECT col FROM tbl GROUP BY cube(col). + * e.g. SELECT col FROM tbl GROUP BY col, SELECT col FROM tbl GROUP BY cube(col). * 2. aggregate function on partition columns with DISTINCT, * e.g. SELECT count(DISTINCT col) FROM tbl GROUP BY col. * 3. aggregate function on partition columns which have same result with DISTINCT keyword. @@ -43,28 +42,33 @@ case class MetadataOnlyOptimizer( catalog: SessionCatalog) extends Rule[LogicalPlan] { private def canSupportMetadataOnly(a: Aggregate): Boolean = { - val aggregateExpressions = a.aggregateExpressions.flatMap { expr => - expr.collect { - case agg: AggregateExpression => agg - } - }.distinct - if (aggregateExpressions.isEmpty) { - // Support for aggregate that has no aggregateFunction when expressions are partition columns - // example: select partitionCol from table group by partitionCol. - // Moreover, multiple-distinct has been rewritted into it by RewriteDistinctAggregates. - true + if (!a.references.forall(_.isPartitionColumn)) { + // Support for scanning only partition columns + false } else { - aggregateExpressions.forall { agg => - if (agg.isDistinct) { - true - } else { - // If function can be evaluated on just the distinct values of a column, it can be used - // by metadata-only optimizer. - agg.aggregateFunction match { - case max: Max => true - case min: Min => true - case hyperLog: HyperLogLogPlusPlus => true - case _ => false + val aggregateExpressions = a.aggregateExpressions.flatMap { expr => + expr.collect { + case agg: AggregateExpression => agg + } + }.distinct + if (aggregateExpressions.isEmpty) { + // Support for aggregate that has no aggregateFunction when expressions are partition + // columns. example: select partitionCol from table group by partitionCol. + // Moreover, multiple-distinct has been rewritted into it by RewriteDistinctAggregates. + true + } else { + aggregateExpressions.forall { agg => + if (agg.isDistinct) { + true + } else { + // If function can be evaluated on just the distinct values of a column, it can be used + // by metadata-only optimizer. + agg.aggregateFunction match { + case max: Max => true + case min: Min => true + case hyperLog: HyperLogLogPlusPlus => true + case _ => false + } } } } @@ -72,7 +76,6 @@ case class MetadataOnlyOptimizer( } private def convertLogicalToMetadataOnly( - project: LogicalPlan, filter: Option[Expression], logical: LogicalRelation, files: HadoopFsRelation): LogicalPlan = { @@ -81,80 +84,62 @@ case class MetadataOnlyOptimizer( attributeMap.getOrElse(field.name, throw new AnalysisException( s"Unable to resolve ${field.name} given [${logical.output.map(_.name).mkString(", ")}]")) } - val projectSet = filter.map(project.references ++ _.references).getOrElse(project.references) - if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val selectedPartitions = files.location.listFiles(filter.map(Seq(_)).getOrElse(Seq.empty)) - val valuesRdd = sparkSession.sparkContext.parallelize(selectedPartitions.map(_.values), 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - valuesPlan - } else { - logical - } + val selectedPartitions = files.location.listFiles(filter.map(Seq(_)).getOrElse(Seq.empty)) + val valuesRdd = sparkSession.sparkContext.parallelize(selectedPartitions.map(_.values), 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + valuesPlan } - private def convertCatalogToMetadataOnly( - project: LogicalPlan, - filter: Option[Expression], - relation: CatalogRelation): LogicalPlan = { + private def convertCatalogToMetadataOnly(relation: CatalogRelation): LogicalPlan = { val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => attributeMap.getOrElse(column, throw new AnalysisException( s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) } - val projectSet = filter.map(project.references ++ _.references).getOrElse(project.references) - if (projectSet.subsetOf(AttributeSet(partitionColumns))) { - val partitionColumnDataTypes = partitionColumns.map(_.dataType) - val partitionValues = catalog.listPartitions(relation.catalogTable.identifier) - .map { p => - InternalRow.fromSeq( - partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { - case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) - }) - } - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - valuesPlan - } else { - relation - } + val partitionColumnDataTypes = partitionColumns.map(_.dataType) + val partitionValues = catalog.listPartitions(relation.catalogTable.identifier) + .map { p => + InternalRow.fromSeq( + partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { + case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) + }) + } + val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) + val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) + valuesPlan } private def convertToMetadataOnly(plan: LogicalPlan): LogicalPlan = plan match { - case p @ Project(fields, child) => + case p @ Project(fields, child) if p.references.forall(_.isPartitionColumn) => child match { case f @ Filter(condition, l @ LogicalRelation(files: HadoopFsRelation, _, _)) - if files.partitionSchema.nonEmpty => - val plan = convertLogicalToMetadataOnly(p, Some(condition), l, files) + if files.partitionSchema.nonEmpty && f.references.forall(_.isPartitionColumn) => + val plan = convertLogicalToMetadataOnly(Some(condition), l, files) p.withNewChildren(f.withNewChildren(plan :: Nil) :: Nil) case l @ LogicalRelation(files: HadoopFsRelation, _, _) if files.partitionSchema.nonEmpty => - val plan = convertLogicalToMetadataOnly(p, None, l, files) + val plan = convertLogicalToMetadataOnly(None, l, files) p.withNewChildren(plan :: Nil) case f @ Filter(condition, relation: CatalogRelation) - if relation.catalogTable.partitionColumnNames.nonEmpty => - val plan = convertCatalogToMetadataOnly(p, Some(condition), relation) + if relation.catalogTable.partitionColumnNames.nonEmpty && + f.references.forall(_.isPartitionColumn) => + val plan = convertCatalogToMetadataOnly(relation) p.withNewChildren(f.withNewChildren(plan :: Nil) :: Nil) case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => - val plan = convertCatalogToMetadataOnly(p, None, relation) + val plan = convertCatalogToMetadataOnly(relation) p.withNewChildren(plan :: Nil) case other => - p.withNewChildren(p.children.map(convertToMetadataOnly(_))) + other } - case f : Filter => - f.withNewChildren(f.children.map(convertToMetadataOnly(_))) - case e: Expand => e.withNewChildren(e.children.map(convertToMetadataOnly(_))) - case u: Union => - u.withNewChildren(u.children.map(convertToMetadataOnly(_))) - case other: LogicalPlan => other } 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 39c8606fd14b..0fa3e440a140 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 @@ -37,7 +37,10 @@ case class LogicalRelation( extends LeafNode with MultiInstanceRelation { override val output: Seq[AttributeReference] = { - val attrs = relation.schema.toAttributes + val attrs = relation.schema.toAttributes.map { attr => + attr.setPartitionColumn( + relation.partitionColumnNames.contains(attr.name.toLowerCase)) + } expectedOutputAttributes.map { expectedAttrs => assert(expectedAttrs.length == attrs.length) attrs.zip(expectedAttrs).map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 06adaf71128a..339512aa2b9b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -151,6 +151,8 @@ case class HadoopFsRelation( }) } + override def partitionColumnNames: Set[String] = partitionSchema.map(_.name.toLowerCase).toSet + def partitionSchemaOption: Option[StructType] = if (partitionSchema.isEmpty) None else Some(partitionSchema) def partitionSpec: PartitionSpec = location.partitionSpec() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index d2077a07f440..53d24051eb27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -185,6 +185,7 @@ trait CreatableRelationProvider { abstract class BaseRelation { def sqlContext: SQLContext def schema: StructType + def partitionColumnNames: Set[String] = Set.empty[String] /** * Returns an estimated size of this relation in bytes. This information is used by the planner 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 58bca2059cac..4ceb0e1933e9 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 @@ -203,22 +203,22 @@ private[hive] case class MetastoreRelation( ) implicit class SchemaAttribute(f: CatalogColumn) { - def toAttribute: AttributeReference = AttributeReference( + def toAttribute(isPartitionColumn: Boolean): AttributeReference = AttributeReference( f.name, CatalystSqlParser.parseDataType(f.dataType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true - )(qualifier = Some(alias.getOrElse(tableName))) + )(qualifier = Some(alias.getOrElse(tableName)), isPartitionColumn = isPartitionColumn) } /** PartitionKey attributes */ - val partitionKeys = catalogTable.partitionColumns.map(_.toAttribute) + val partitionKeys = catalogTable.partitionColumns.map(_.toAttribute(true)) /** Non-partitionKey attributes */ // TODO: just make this hold the schema itself, not just non-partition columns val attributes = catalogTable.schema .filter { c => !catalogTable.partitionColumnNames.contains(c.name) } - .map(_.toAttribute) + .map(_.toAttribute(false)) val output = attributes ++ partitionKeys 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 f17d0ce55d80..dc7a6867c3cd 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 @@ -1701,21 +1701,28 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |select key, value from data_15752 """.stripMargin) } - checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) checkAnswer( sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), Row(null) :: Row(12) :: Nil) - checkAnswer( - sql("select hr from (select hr from srcpart_15752 where hr = 11 union all " + - "select hr from srcpart_15752 where hr= 12)t group by hr"), - Row(11) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) checkAnswer(sql("select distinct hr from srcpart_15752"), Row(11) :: Row(12) :: Nil) checkAnswer(sql("select distinct hr from srcpart_15752 where hr = 11"), Row(11)) + // Now donot support metadata only optimizer + checkAnswer( + sql("select hr from (select hr from srcpart_15752 where hr = 11 union all " + + "select hr from srcpart_15752 where hr= 12)t group by hr"), + Row(11) :: Row(12) :: Nil) + checkAnswer( + sql("select max(x) from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + Row(13)) + checkAnswer( + sql("select distinct x from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + Row(13)) + sql( """ |CREATE TABLE srctext_15752 (key INT, value STRING) @@ -1728,26 +1735,15 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |select key, value from data_15752 """.stripMargin) } - checkAnswer(sql("select hr from srctext_15752 where hr = 11 group by hr"), Row(11)) checkAnswer( sql("select hr from srctext_15752 where hr = 12 group by rollup(hr)"), Row(null) :: Row(12) :: Nil) - checkAnswer( - sql("select hr from (select hr from srctext_15752 where hr = 11 union all " + - "select hr from srctext_15752 where hr= 12)t group by hr"), - Row(11) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) - checkAnswer( - sql("select max(x) from (select hr + 1 as x from srctext_15752 where hr = 12) t"), - Row(13)) checkAnswer(sql("select distinct hr from srctext_15752"), Row(11) :: Row(12) :: Nil) checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) - checkAnswer( - sql("select distinct x from (select hr + 1 as x from srctext_15752 where hr = 12) t"), - Row(13)) } } } From a4045ca3e492d76ae6f7d919cd1130bc9d1a52c5 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 29 Jun 2016 20:05:47 +0800 Subject: [PATCH 20/45] add comments --- .../apache/spark/sql/execution/MetadataOnlyOptimizer.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 9709a9871d51..9fcabccfa463 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -109,6 +109,11 @@ case class MetadataOnlyOptimizer( valuesPlan } + /** + * When scanning only partition columns, convert LogicalRelation or CatalogRelation to LogicalRDD. + * Now support logical plan: + * Aggregate [Expand] Project [Filter] (LogicalRelation | CatalogRelation) + */ private def convertToMetadataOnly(plan: LogicalPlan): LogicalPlan = plan match { case p @ Project(fields, child) if p.references.forall(_.isPartitionColumn) => child match { From 0a023e7ce7a20af0b25d320e7fd4455f9e1b3029 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 29 Jun 2016 20:17:31 +0800 Subject: [PATCH 21/45] fix minor --- .../scala/org/apache/spark/sql/catalyst/catalog/interface.scala | 2 +- .../org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala | 2 +- 2 files changed, 2 insertions(+), 2 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 43c5e7b2992b..6197acab3378 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 @@ -231,7 +231,7 @@ object CatalogTypes { * If we can in the future consolidate SimpleCatalogRelation and MetastoreRelation, we should * probably remove this interface. */ -trait CatalogRelation extends LogicalPlan { +trait CatalogRelation { def catalogTable: CatalogTable def output: Seq[Attribute] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 9fcabccfa463..d37bfa923c4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -145,7 +145,7 @@ case class MetadataOnlyOptimizer( case e: Expand => e.withNewChildren(e.children.map(convertToMetadataOnly(_))) - case other: LogicalPlan => + case other => other } From a9b38abf0972bd956142278379e2e2e546c2b1a1 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 29 Jun 2016 22:47:05 +0800 Subject: [PATCH 22/45] rename --- .../sql/catalyst/analysis/Analyzer.scala | 4 +-- .../expressions/namedExpressions.scala | 34 +++++++++---------- .../sql/execution/MetadataOnlyOptimizer.scala | 8 ++--- .../datasources/LogicalRelation.scala | 2 +- .../spark/sql/hive/MetastoreRelation.scala | 4 +-- 5 files changed, 26 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 044121ea45a5..65457a4fa58d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -293,14 +293,14 @@ class Analyzer( // Ensure all the expressions have been resolved. case x: GroupingSets if x.expressions.forall(_.resolved) => val gid = AttributeReference(VirtualColumn.groupingIdName, IntegerType, false)( - isPartitionColumn = true) + isMetadataColumn = true) // Expand works by setting grouping expressions to null as determined by the bitmasks. To // prevent these null values from being used in an aggregate instead of the original value // we need to create new aliases for all group by expressions that will only be used for // the intended purpose. val groupByAliases: Seq[Alias] = x.groupByExprs.map { - case e: NamedExpression => Alias(e, e.name)(isPartitionColumn = e.isPartitionColumn) + case e: NamedExpression => Alias(e, e.name)(isMetadataColumn = e.isMetadataColumn) case other => Alias(other, other.toString)() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index ba7237dd1825..2816383e9a42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -83,8 +83,8 @@ trait NamedExpression extends Expression { /** Returns true if the expression is generated by Catalyst */ def isGenerated: java.lang.Boolean = false - /** Returns true if the expression is a partition column */ - def isPartitionColumn: java.lang.Boolean = false + /** Returns true if the expression is a metadata column */ + def isMetadataColumn: java.lang.Boolean = false /** Returns a copy of this expression with a new `exprId`. */ def newInstance(): NamedExpression @@ -130,14 +130,14 @@ abstract class Attribute extends LeafExpression with NamedExpression with NullIn * tableName and subQueryAlias are possible qualifiers. * @param explicitMetadata Explicit metadata associated with this alias that overwrites child's. * @param isGenerated A flag to indicate if this alias is generated by Catalyst - * @param isPartitionColumn A flag to indicate if this alias is a partition column + * @param isMetadataColumn A flag to indicate if this alias is a metadata column */ case class Alias(child: Expression, name: String)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, val explicitMetadata: Option[Metadata] = None, override val isGenerated: java.lang.Boolean = false, - override val isPartitionColumn: java.lang.Boolean = false) + override val isMetadataColumn: java.lang.Boolean = false) extends UnaryExpression with NamedExpression { // Alias(Generator, xx) need to be transformed into Generate(generator, ...) @@ -164,12 +164,12 @@ case class Alias(child: Expression, name: String)( def newInstance(): NamedExpression = Alias(child, name)( qualifier = qualifier, explicitMetadata = explicitMetadata, isGenerated = isGenerated, - isPartitionColumn = isPartitionColumn) + isMetadataColumn = isMetadataColumn) override def toAttribute: Attribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable, metadata)( - exprId, qualifier, isGenerated, isPartitionColumn) + exprId, qualifier, isGenerated, isMetadataColumn) } else { UnresolvedAttribute(name) } @@ -178,7 +178,7 @@ case class Alias(child: Expression, name: String)( override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: explicitMetadata :: isGenerated :: isPartitionColumn :: Nil + exprId :: qualifier :: explicitMetadata :: isGenerated :: isMetadataColumn :: Nil } override def hashCode(): Int = { @@ -212,7 +212,7 @@ case class Alias(child: Expression, name: String)( * qualified way. Consider the examples tableName.name, subQueryAlias.name. * tableName and subQueryAlias are possible qualifiers. * @param isGenerated A flag to indicate if this reference is generated by Catalyst - * @param isPartitionColumn A flag to indicate if this reference is a partition column + * @param isMetadataColumn A flag to indicate if this reference is a metadata column */ case class AttributeReference( name: String, @@ -222,7 +222,7 @@ case class AttributeReference( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, override val isGenerated: java.lang.Boolean = false, - override val isPartitionColumn: java.lang.Boolean = false) + override val isMetadataColumn: java.lang.Boolean = false) extends Attribute with Unevaluable { /** @@ -260,7 +260,7 @@ case class AttributeReference( override def newInstance(): AttributeReference = AttributeReference(name, dataType, nullable, metadata)( - qualifier = qualifier, isGenerated = isGenerated, isPartitionColumn = isPartitionColumn) + qualifier = qualifier, isGenerated = isGenerated, isMetadataColumn = isMetadataColumn) /** * Returns a copy of this [[AttributeReference]] with changed nullability. @@ -270,7 +270,7 @@ case class AttributeReference( this } else { AttributeReference(name, dataType, newNullability, metadata)( - exprId, qualifier, isGenerated, isPartitionColumn) + exprId, qualifier, isGenerated, isMetadataColumn) } } @@ -279,7 +279,7 @@ case class AttributeReference( this } else { AttributeReference(newName, dataType, nullable, metadata)( - exprId, qualifier, isGenerated, isPartitionColumn) + exprId, qualifier, isGenerated, isMetadataColumn) } } @@ -291,7 +291,7 @@ case class AttributeReference( this } else { AttributeReference(name, dataType, nullable, metadata)( - exprId, newQualifier, isGenerated, isPartitionColumn) + exprId, newQualifier, isGenerated, isMetadataColumn) } } @@ -300,17 +300,17 @@ case class AttributeReference( this } else { AttributeReference(name, dataType, nullable, metadata)( - newExprId, qualifier, isGenerated, isPartitionColumn) + newExprId, qualifier, isGenerated, isMetadataColumn) } } - def setPartitionColumn(isPartitionColumn: Boolean): AttributeReference = { + def withMetadataColumn(isMetadataColumn: Boolean): AttributeReference = { AttributeReference(name, dataType, nullable, metadata)( - exprId, qualifier, isGenerated, isPartitionColumn) + exprId, qualifier, isGenerated, isMetadataColumn) } override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: isGenerated :: isPartitionColumn :: Nil + exprId :: qualifier :: isGenerated :: isMetadataColumn :: Nil } override def toString: String = s"$name#${exprId.id}$typeSuffix" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index d37bfa923c4c..0a151e1f33a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -42,7 +42,7 @@ case class MetadataOnlyOptimizer( catalog: SessionCatalog) extends Rule[LogicalPlan] { private def canSupportMetadataOnly(a: Aggregate): Boolean = { - if (!a.references.forall(_.isPartitionColumn)) { + if (!a.references.forall(_.isMetadataColumn)) { // Support for scanning only partition columns false } else { @@ -115,10 +115,10 @@ case class MetadataOnlyOptimizer( * Aggregate [Expand] Project [Filter] (LogicalRelation | CatalogRelation) */ private def convertToMetadataOnly(plan: LogicalPlan): LogicalPlan = plan match { - case p @ Project(fields, child) if p.references.forall(_.isPartitionColumn) => + case p @ Project(fields, child) if p.references.forall(_.isMetadataColumn) => child match { case f @ Filter(condition, l @ LogicalRelation(files: HadoopFsRelation, _, _)) - if files.partitionSchema.nonEmpty && f.references.forall(_.isPartitionColumn) => + if files.partitionSchema.nonEmpty && f.references.forall(_.isMetadataColumn) => val plan = convertLogicalToMetadataOnly(Some(condition), l, files) p.withNewChildren(f.withNewChildren(plan :: Nil) :: Nil) @@ -129,7 +129,7 @@ case class MetadataOnlyOptimizer( case f @ Filter(condition, relation: CatalogRelation) if relation.catalogTable.partitionColumnNames.nonEmpty && - f.references.forall(_.isPartitionColumn) => + f.references.forall(_.isMetadataColumn) => val plan = convertCatalogToMetadataOnly(relation) p.withNewChildren(f.withNewChildren(plan :: Nil) :: 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 0fa3e440a140..2d5f6d90f367 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 @@ -38,7 +38,7 @@ case class LogicalRelation( override val output: Seq[AttributeReference] = { val attrs = relation.schema.toAttributes.map { attr => - attr.setPartitionColumn( + attr.withMetadataColumn( relation.partitionColumnNames.contains(attr.name.toLowerCase)) } expectedOutputAttributes.map { expectedAttrs => 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 4ceb0e1933e9..68c0c985b56a 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 @@ -203,12 +203,12 @@ private[hive] case class MetastoreRelation( ) implicit class SchemaAttribute(f: CatalogColumn) { - def toAttribute(isPartitionColumn: Boolean): AttributeReference = AttributeReference( + def toAttribute(isMetadataColumn: Boolean): AttributeReference = AttributeReference( f.name, CatalystSqlParser.parseDataType(f.dataType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true - )(qualifier = Some(alias.getOrElse(tableName)), isPartitionColumn = isPartitionColumn) + )(qualifier = Some(alias.getOrElse(tableName)), isMetadataColumn = isMetadataColumn) } /** PartitionKey attributes */ From a5ea995929358ab999f0636d053901a9af84a548 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 30 Jun 2016 02:24:59 +0800 Subject: [PATCH 23/45] update --- .../spark/sql/catalyst/CatalystConf.scala | 2 ++ .../sql/execution/MetadataOnlyOptimizer.scala | 22 ++++++++----------- .../spark/sql/execution/SparkOptimizer.scala | 3 +-- .../spark/sql/internal/SessionState.scala | 3 +-- 4 files changed, 13 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index 4df100c2a830..cabef8d29782 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -28,6 +28,7 @@ trait CatalystConf { def orderByOrdinal: Boolean def groupByOrdinal: Boolean + def optimizerMetadataOnly: Boolean def optimizerMaxIterations: Int def optimizerInSetConversionThreshold: Int def maxCaseBranchesForCodegen: Int @@ -51,6 +52,7 @@ case class SimpleCatalystConf( caseSensitiveAnalysis: Boolean, orderByOrdinal: Boolean = true, groupByOrdinal: Boolean = true, + optimizerMetadataOnly: Boolean = true, optimizerMaxIterations: Int = 100, optimizerInSetConversionThreshold: Int = 10, maxCaseBranchesForCodegen: Int = 20, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index 0a151e1f33a2..e3299fbc2ba8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.{CatalystConf, InternalRow} import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -38,8 +38,8 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela * e.g. SELECT Max(col2) FROM tbl GROUP BY col1. */ case class MetadataOnlyOptimizer( - sparkSession: SparkSession, - catalog: SessionCatalog) extends Rule[LogicalPlan] { + catalog: SessionCatalog, + conf: CatalystConf) extends Rule[LogicalPlan] { private def canSupportMetadataOnly(a: Aggregate): Boolean = { if (!a.references.forall(_.isMetadataColumn)) { @@ -85,9 +85,7 @@ case class MetadataOnlyOptimizer( s"Unable to resolve ${field.name} given [${logical.output.map(_.name).mkString(", ")}]")) } val selectedPartitions = files.location.listFiles(filter.map(Seq(_)).getOrElse(Seq.empty)) - val valuesRdd = sparkSession.sparkContext.parallelize(selectedPartitions.map(_.values), 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - valuesPlan + LocalRelation(partitionColumns, selectedPartitions.map(_.values)) } private def convertCatalogToMetadataOnly(relation: CatalogRelation): LogicalPlan = { @@ -104,14 +102,12 @@ case class MetadataOnlyOptimizer( case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) }) } - val valuesRdd = sparkSession.sparkContext.parallelize(partitionValues, 1) - val valuesPlan = LogicalRDD(partitionColumns, valuesRdd)(sparkSession) - valuesPlan + LocalRelation(partitionColumns, partitionValues) } /** - * When scanning only partition columns, convert LogicalRelation or CatalogRelation to LogicalRDD. - * Now support logical plan: + * When scanning only partition columns, convert LogicalRelation or CatalogRelation + * to LocalRelation. Now support logical plan: * Aggregate [Expand] Project [Filter] (LogicalRelation | CatalogRelation) */ private def convertToMetadataOnly(plan: LogicalPlan): LogicalPlan = plan match { @@ -150,7 +146,7 @@ case class MetadataOnlyOptimizer( } def apply(plan: LogicalPlan): LogicalPlan = { - if (!sparkSession.sessionState.conf.optimizerMetadataOnly) { + if (!conf.optimizerMetadataOnly) { return plan } plan.transform { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 9a680bdac962..d61f48e359a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -24,14 +24,13 @@ import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate import org.apache.spark.sql.internal.SQLConf class SparkOptimizer( - sparkSession: SparkSession, catalog: SessionCatalog, conf: SQLConf, experimentalMethods: ExperimentalMethods) extends Optimizer(catalog, conf) { override def batches: Seq[Batch] = super.batches :+ - Batch("Metadata Only Optimization", Once, MetadataOnlyOptimizer(sparkSession, catalog)) :+ + Batch("Metadata Only Optimization", Once, MetadataOnlyOptimizer(catalog, conf)) :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } 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 f10723618380..5f5cf5c6d30c 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 @@ -123,8 +123,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { /** * Logical query plan optimizer. */ - lazy val optimizer: Optimizer = - new SparkOptimizer(sparkSession, catalog, conf, experimentalMethods) + lazy val optimizer: Optimizer = new SparkOptimizer(catalog, conf, experimentalMethods) /** * Parser that extracts expressions, plans, table identifiers etc. from SQL texts. From 1bed08deb2eba2426b83bd5f02dcb0a2ce18afd6 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 30 Jun 2016 02:27:12 +0800 Subject: [PATCH 24/45] fix monir --- .../scala/org/apache/spark/sql/execution/SparkOptimizer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index d61f48e359a4..b78390fb5d86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{ExperimentalMethods, SparkSession} +import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate From a22e9626e6294671e0915822def6eb283a72a643 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 1 Jul 2016 16:36:04 +0800 Subject: [PATCH 25/45] refactor --- .../sql/catalyst/analysis/Analyzer.scala | 5 +- .../expressions/namedExpressions.scala | 37 ++-- .../sql/execution/MetadataOnlyOptimizer.scala | 186 ++++++++---------- .../datasources/LogicalRelation.scala | 5 +- .../datasources/fileSourceInterfaces.scala | 2 - .../apache/spark/sql/sources/interfaces.scala | 1 - .../spark/sql/hive/MetastoreRelation.scala | 8 +- .../sql/hive/execution/SQLQuerySuite.scala | 27 +-- 8 files changed, 115 insertions(+), 156 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 65457a4fa58d..96f2e38946f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -292,15 +292,14 @@ class Analyzer( // Ensure all the expressions have been resolved. case x: GroupingSets if x.expressions.forall(_.resolved) => - val gid = AttributeReference(VirtualColumn.groupingIdName, IntegerType, false)( - isMetadataColumn = true) + val gid = AttributeReference(VirtualColumn.groupingIdName, IntegerType, false)() // Expand works by setting grouping expressions to null as determined by the bitmasks. To // prevent these null values from being used in an aggregate instead of the original value // we need to create new aliases for all group by expressions that will only be used for // the intended purpose. val groupByAliases: Seq[Alias] = x.groupByExprs.map { - case e: NamedExpression => Alias(e, e.name)(isMetadataColumn = e.isMetadataColumn) + case e: NamedExpression => Alias(e, e.name)() case other => Alias(other, other.toString)() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 2816383e9a42..9987c950b88a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -83,9 +83,6 @@ trait NamedExpression extends Expression { /** Returns true if the expression is generated by Catalyst */ def isGenerated: java.lang.Boolean = false - /** Returns true if the expression is a metadata column */ - def isMetadataColumn: java.lang.Boolean = false - /** Returns a copy of this expression with a new `exprId`. */ def newInstance(): NamedExpression @@ -130,14 +127,12 @@ abstract class Attribute extends LeafExpression with NamedExpression with NullIn * tableName and subQueryAlias are possible qualifiers. * @param explicitMetadata Explicit metadata associated with this alias that overwrites child's. * @param isGenerated A flag to indicate if this alias is generated by Catalyst - * @param isMetadataColumn A flag to indicate if this alias is a metadata column */ case class Alias(child: Expression, name: String)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, val explicitMetadata: Option[Metadata] = None, - override val isGenerated: java.lang.Boolean = false, - override val isMetadataColumn: java.lang.Boolean = false) + override val isGenerated: java.lang.Boolean = false) extends UnaryExpression with NamedExpression { // Alias(Generator, xx) need to be transformed into Generate(generator, ...) @@ -163,13 +158,12 @@ case class Alias(child: Expression, name: String)( def newInstance(): NamedExpression = Alias(child, name)( - qualifier = qualifier, explicitMetadata = explicitMetadata, isGenerated = isGenerated, - isMetadataColumn = isMetadataColumn) + qualifier = qualifier, explicitMetadata = explicitMetadata, isGenerated = isGenerated) override def toAttribute: Attribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable, metadata)( - exprId, qualifier, isGenerated, isMetadataColumn) + exprId, qualifier, isGenerated) } else { UnresolvedAttribute(name) } @@ -178,7 +172,7 @@ case class Alias(child: Expression, name: String)( override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: explicitMetadata :: isGenerated :: isMetadataColumn :: Nil + exprId :: qualifier :: explicitMetadata :: isGenerated :: Nil } override def hashCode(): Int = { @@ -212,7 +206,6 @@ case class Alias(child: Expression, name: String)( * qualified way. Consider the examples tableName.name, subQueryAlias.name. * tableName and subQueryAlias are possible qualifiers. * @param isGenerated A flag to indicate if this reference is generated by Catalyst - * @param isMetadataColumn A flag to indicate if this reference is a metadata column */ case class AttributeReference( name: String, @@ -221,8 +214,7 @@ case class AttributeReference( override val metadata: Metadata = Metadata.empty)( val exprId: ExprId = NamedExpression.newExprId, val qualifier: Option[String] = None, - override val isGenerated: java.lang.Boolean = false, - override val isMetadataColumn: java.lang.Boolean = false) + override val isGenerated: java.lang.Boolean = false) extends Attribute with Unevaluable { /** @@ -260,7 +252,7 @@ case class AttributeReference( override def newInstance(): AttributeReference = AttributeReference(name, dataType, nullable, metadata)( - qualifier = qualifier, isGenerated = isGenerated, isMetadataColumn = isMetadataColumn) + qualifier = qualifier, isGenerated = isGenerated) /** * Returns a copy of this [[AttributeReference]] with changed nullability. @@ -269,8 +261,7 @@ case class AttributeReference( if (nullable == newNullability) { this } else { - AttributeReference(name, dataType, newNullability, metadata)( - exprId, qualifier, isGenerated, isMetadataColumn) + AttributeReference(name, dataType, newNullability, metadata)(exprId, qualifier, isGenerated) } } @@ -278,8 +269,7 @@ case class AttributeReference( if (name == newName) { this } else { - AttributeReference(newName, dataType, nullable, metadata)( - exprId, qualifier, isGenerated, isMetadataColumn) + AttributeReference(newName, dataType, nullable, metadata)(exprId, qualifier, isGenerated) } } @@ -290,8 +280,7 @@ case class AttributeReference( if (newQualifier == qualifier) { this } else { - AttributeReference(name, dataType, nullable, metadata)( - exprId, newQualifier, isGenerated, isMetadataColumn) + AttributeReference(name, dataType, nullable, metadata)(exprId, newQualifier, isGenerated) } } @@ -299,18 +288,16 @@ case class AttributeReference( if (exprId == newExprId) { this } else { - AttributeReference(name, dataType, nullable, metadata)( - newExprId, qualifier, isGenerated, isMetadataColumn) + AttributeReference(name, dataType, nullable, metadata)(newExprId, qualifier, isGenerated) } } def withMetadataColumn(isMetadataColumn: Boolean): AttributeReference = { - AttributeReference(name, dataType, nullable, metadata)( - exprId, qualifier, isGenerated, isMetadataColumn) + AttributeReference(name, dataType, nullable, metadata)(exprId, qualifier, isGenerated) } override protected final def otherCopyArgs: Seq[AnyRef] = { - exprId :: qualifier :: isGenerated :: isMetadataColumn :: Nil + exprId :: qualifier :: isGenerated :: Nil } override def toString: String = s"$name#${exprId.id}$typeSuffix" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index e3299fbc2ba8..ff500add2d9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -27,131 +27,107 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} /** - * When scanning only partition columns, get results based on metadata without scanning files. + * When scanning only partition columns, get results based on partition data without scanning files. * It's used for operators that only need distinct values. Currently only [[Aggregate]] operator * which satisfy the following conditions are supported: - * 1. aggregate expression is partition columns, - * e.g. SELECT col FROM tbl GROUP BY col, SELECT col FROM tbl GROUP BY cube(col). - * 2. aggregate function on partition columns with DISTINCT, + * 1. aggregate expression is partition columns. + * e.g. SELECT col FROM tbl GROUP BY col. + * 2. aggregate function on partition columns with DISTINCT. * e.g. SELECT count(DISTINCT col) FROM tbl GROUP BY col. - * 3. aggregate function on partition columns which have same result with DISTINCT keyword. + * 3. aggregate function on partition columns which have same result w or w/o DISTINCT keyword. * e.g. SELECT Max(col2) FROM tbl GROUP BY col1. */ case class MetadataOnlyOptimizer( - catalog: SessionCatalog, - conf: CatalystConf) extends Rule[LogicalPlan] { + catalog: SessionCatalog, + conf: CatalystConf) extends Rule[LogicalPlan] { - private def canSupportMetadataOnly(a: Aggregate): Boolean = { - if (!a.references.forall(_.isMetadataColumn)) { - // Support for scanning only partition columns - false - } else { - val aggregateExpressions = a.aggregateExpressions.flatMap { expr => - expr.collect { - case agg: AggregateExpression => agg - } - }.distinct - if (aggregateExpressions.isEmpty) { - // Support for aggregate that has no aggregateFunction when expressions are partition - // columns. example: select partitionCol from table group by partitionCol. - // Moreover, multiple-distinct has been rewritted into it by RewriteDistinctAggregates. - true - } else { - aggregateExpressions.forall { agg => - if (agg.isDistinct) { - true - } else { - // If function can be evaluated on just the distinct values of a column, it can be used - // by metadata-only optimizer. - agg.aggregateFunction match { - case max: Max => true - case min: Min => true - case hyperLog: HyperLogLogPlusPlus => true + def apply(plan: LogicalPlan): LogicalPlan = { + if (!conf.optimizerMetadataOnly) { + return plan + } + + plan.transform { + case a @ Aggregate(_, aggExprs, child @ PartitionedRelation(partAttrs, relation)) => + if (a.references.subsetOf(partAttrs)) { + val aggFunctions = aggExprs.flatMap(_.collect { + case agg: AggregateExpression => agg + }) + val isPartitionDataOnly = aggFunctions.isEmpty || aggFunctions.forall { agg => + agg.isDistinct || (agg.aggregateFunction match { + case _: Max => true + case _: Min => true case _ => false - } + }) + } + if (isPartitionDataOnly) { + a.withNewChildren(Seq(usePartitionData(child, relation))) + } else { + a } + } else { + a } - } } } - private def convertLogicalToMetadataOnly( - filter: Option[Expression], - logical: LogicalRelation, - files: HadoopFsRelation): LogicalPlan = { - val attributeMap = logical.output.map(attr => (attr.name, attr)).toMap - val partitionColumns = files.partitionSchema.map { field => - attributeMap.getOrElse(field.name, throw new AnalysisException( - s"Unable to resolve ${field.name} given [${logical.output.map(_.name).mkString(", ")}]")) - } - val selectedPartitions = files.location.listFiles(filter.map(Seq(_)).getOrElse(Seq.empty)) - LocalRelation(partitionColumns, selectedPartitions.map(_.values)) - } + private def usePartitionData(child: LogicalPlan, relation: LogicalPlan): LogicalPlan = { + child transform { + case plan if plan eq relation => + relation match { + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) => + val partColumns = fsRelation.partitionSchema.map(_.name.toLowerCase).toSet + val partAttrs = l.output.filter(a => partColumns.contains(a.name.toLowerCase)) + val partitionData = fsRelation.location.listFiles(Nil) + LocalRelation(partAttrs, partitionData.map(_.values)) + + case relation: CatalogRelation => + val partColumns = relation.catalogTable.partitionColumnNames.map(_.toLowerCase).toSet + val partAttrs = relation.output.filter(a => partColumns.contains(a.name.toLowerCase)) + val partitionData = catalog.listPartitions(relation.catalogTable.identifier).map { p => + InternalRow.fromSeq(partAttrs.map { attr => + Cast(Literal(p.spec(attr.name)), attr.dataType).eval() + }) + } + LocalRelation(partAttrs, partitionData) - private def convertCatalogToMetadataOnly(relation: CatalogRelation): LogicalPlan = { - val attributeMap = relation.output.map(attr => (attr.name, attr)).toMap - val partitionColumns = relation.catalogTable.partitionColumnNames.map { column => - attributeMap.getOrElse(column, throw new AnalysisException( - s"Unable to resolve ${column} given [${relation.output.map(_.name).mkString(", ")}]")) + case _ => throw new IllegalStateException() + } } - val partitionColumnDataTypes = partitionColumns.map(_.dataType) - val partitionValues = catalog.listPartitions(relation.catalogTable.identifier) - .map { p => - InternalRow.fromSeq( - partitionColumns.map(a => p.spec(a.name)).zip(partitionColumnDataTypes).map { - case (rawValue, dataType) => Cast(Literal(rawValue), dataType).eval(null) - }) - } - LocalRelation(partitionColumns, partitionValues) } - /** - * When scanning only partition columns, convert LogicalRelation or CatalogRelation - * to LocalRelation. Now support logical plan: - * Aggregate [Expand] Project [Filter] (LogicalRelation | CatalogRelation) - */ - private def convertToMetadataOnly(plan: LogicalPlan): LogicalPlan = plan match { - case p @ Project(fields, child) if p.references.forall(_.isMetadataColumn) => - child match { - case f @ Filter(condition, l @ LogicalRelation(files: HadoopFsRelation, _, _)) - if files.partitionSchema.nonEmpty && f.references.forall(_.isMetadataColumn) => - val plan = convertLogicalToMetadataOnly(Some(condition), l, files) - p.withNewChildren(f.withNewChildren(plan :: Nil) :: Nil) - - case l @ LogicalRelation(files: HadoopFsRelation, _, _) - if files.partitionSchema.nonEmpty => - val plan = convertLogicalToMetadataOnly(None, l, files) - p.withNewChildren(plan :: Nil) - - case f @ Filter(condition, relation: CatalogRelation) - if relation.catalogTable.partitionColumnNames.nonEmpty && - f.references.forall(_.isMetadataColumn) => - val plan = convertCatalogToMetadataOnly(relation) - p.withNewChildren(f.withNewChildren(plan :: Nil) :: Nil) + object PartitionedRelation { + def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { + case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) + if fsRelation.partitionSchema.nonEmpty => + val partColumns = fsRelation.partitionSchema.map(_.name.toLowerCase).toSet + val partAttrs = l.output.filter(a => partColumns.contains(a.name.toLowerCase)) + Some(AttributeSet(partAttrs), l) - case relation: CatalogRelation - if relation.catalogTable.partitionColumnNames.nonEmpty => - val plan = convertCatalogToMetadataOnly(relation) - p.withNewChildren(plan :: Nil) + case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => + val partColumns = relation.catalogTable.partitionColumnNames.map(_.toLowerCase).toSet + val partAttrs = relation.output.filter(a => partColumns.contains(a.name.toLowerCase)) + Some(AttributeSet(partAttrs), relation) - case other => - other - } - - case e: Expand => - e.withNewChildren(e.children.map(convertToMetadataOnly(_))) - - case other => - other - } + case p @ Project(projectList, child) if projectList.forall(_.deterministic) => + unapply(child).flatMap { + case (partAttrs, relation) => + if (p.references.subsetOf(partAttrs)) { + Some(p.outputSet, relation) + } else { + None + } + } - def apply(plan: LogicalPlan): LogicalPlan = { - if (!conf.optimizerMetadataOnly) { - return plan - } - plan.transform { - case a @ Aggregate(_, _, child) if canSupportMetadataOnly(a) => - a.withNewChildren(convertToMetadataOnly(child) :: Nil) + case f @ Filter(condition, child) if condition.deterministic => + unapply(child).flatMap { + case (partAttrs, relation) => + if (f.references.subsetOf(partAttrs)) { + Some(f.outputSet, relation) + } else { + None + } + } + case _ => None } } } 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 2d5f6d90f367..39c8606fd14b 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 @@ -37,10 +37,7 @@ case class LogicalRelation( extends LeafNode with MultiInstanceRelation { override val output: Seq[AttributeReference] = { - val attrs = relation.schema.toAttributes.map { attr => - attr.withMetadataColumn( - relation.partitionColumnNames.contains(attr.name.toLowerCase)) - } + val attrs = relation.schema.toAttributes expectedOutputAttributes.map { expectedAttrs => assert(expectedAttrs.length == attrs.length) attrs.zip(expectedAttrs).map { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala index 339512aa2b9b..06adaf71128a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala @@ -151,8 +151,6 @@ case class HadoopFsRelation( }) } - override def partitionColumnNames: Set[String] = partitionSchema.map(_.name.toLowerCase).toSet - def partitionSchemaOption: Option[StructType] = if (partitionSchema.isEmpty) None else Some(partitionSchema) def partitionSpec: PartitionSpec = location.partitionSpec() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 53d24051eb27..d2077a07f440 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -185,7 +185,6 @@ trait CreatableRelationProvider { abstract class BaseRelation { def sqlContext: SQLContext def schema: StructType - def partitionColumnNames: Set[String] = Set.empty[String] /** * Returns an estimated size of this relation in bytes. This information is used by the planner 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 68c0c985b56a..26d880fdc45f 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 @@ -203,22 +203,22 @@ private[hive] case class MetastoreRelation( ) implicit class SchemaAttribute(f: CatalogColumn) { - def toAttribute(isMetadataColumn: Boolean): AttributeReference = AttributeReference( + def toAttribute(): AttributeReference = AttributeReference( f.name, CatalystSqlParser.parseDataType(f.dataType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true - )(qualifier = Some(alias.getOrElse(tableName)), isMetadataColumn = isMetadataColumn) + )(qualifier = Some(alias.getOrElse(tableName))) } /** PartitionKey attributes */ - val partitionKeys = catalogTable.partitionColumns.map(_.toAttribute(true)) + val partitionKeys = catalogTable.partitionColumns.map(_.toAttribute()) /** Non-partitionKey attributes */ // TODO: just make this hold the schema itself, not just non-partition columns val attributes = catalogTable.schema .filter { c => !catalogTable.partitionColumnNames.contains(c.name) } - .map(_.toAttribute(false)) + .map(_.toAttribute()) val output = attributes ++ partitionKeys 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 dc7a6867c3cd..00a933921808 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 @@ -1702,26 +1702,26 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) } checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) - checkAnswer( - sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), - Row(null) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) + checkAnswer( + sql("select max(x) from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + Row(13)) checkAnswer(sql("select distinct hr from srcpart_15752"), Row(11) :: Row(12) :: Nil) checkAnswer(sql("select distinct hr from srcpart_15752 where hr = 11"), Row(11)) + checkAnswer( + sql("select distinct x from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + Row(13)) // Now donot support metadata only optimizer + checkAnswer( + sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), + Row(null) :: Row(12) :: Nil) checkAnswer( sql("select hr from (select hr from srcpart_15752 where hr = 11 union all " + "select hr from srcpart_15752 where hr= 12)t group by hr"), Row(11) :: Row(12) :: Nil) - checkAnswer( - sql("select max(x) from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), - Row(13)) - checkAnswer( - sql("select distinct x from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), - Row(13)) sql( """ @@ -1736,14 +1736,17 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) } checkAnswer(sql("select hr from srctext_15752 where hr = 11 group by hr"), Row(11)) - checkAnswer( - sql("select hr from srctext_15752 where hr = 12 group by rollup(hr)"), - Row(null) :: Row(12) :: Nil) checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) + checkAnswer( + sql("select max(x) from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + Row(13)) checkAnswer(sql("select distinct hr from srctext_15752"), Row(11) :: Row(12) :: Nil) checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) + checkAnswer( + sql("select distinct x from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + Row(13)) } } } From 41fef2c40f4929fd26476ecdfa3ee8160394a7d3 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 1 Jul 2016 16:39:36 +0800 Subject: [PATCH 26/45] update --- .../spark/sql/catalyst/expressions/namedExpressions.scala | 4 ---- .../scala/org/apache/spark/sql/hive/MetastoreRelation.scala | 6 +++--- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 9987c950b88a..306a99d5a37b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -292,10 +292,6 @@ case class AttributeReference( } } - def withMetadataColumn(isMetadataColumn: Boolean): AttributeReference = { - AttributeReference(name, dataType, nullable, metadata)(exprId, qualifier, isGenerated) - } - override protected final def otherCopyArgs: Seq[AnyRef] = { exprId :: qualifier :: isGenerated :: Nil } 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 26d880fdc45f..58bca2059cac 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 @@ -203,7 +203,7 @@ private[hive] case class MetastoreRelation( ) implicit class SchemaAttribute(f: CatalogColumn) { - def toAttribute(): AttributeReference = AttributeReference( + def toAttribute: AttributeReference = AttributeReference( f.name, CatalystSqlParser.parseDataType(f.dataType), // Since data can be dumped in randomly with no validation, everything is nullable. @@ -212,13 +212,13 @@ private[hive] case class MetastoreRelation( } /** PartitionKey attributes */ - val partitionKeys = catalogTable.partitionColumns.map(_.toAttribute()) + val partitionKeys = catalogTable.partitionColumns.map(_.toAttribute) /** Non-partitionKey attributes */ // TODO: just make this hold the schema itself, not just non-partition columns val attributes = catalogTable.schema .filter { c => !catalogTable.partitionColumnNames.contains(c.name) } - .map(_.toAttribute()) + .map(_.toAttribute) val output = attributes ++ partitionKeys From 88f7308173829ca2473690a0c409c438d3cd5cf4 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 1 Jul 2016 16:56:11 +0800 Subject: [PATCH 27/45] update --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 ++ 1 file changed, 2 insertions(+) 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 552eacfcbceb..2e0b5d59b578 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.execution.command.CreateDataSourceTableUtils._ @@ -36,6 +37,7 @@ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Pa import org.apache.spark.sql.hive.orc.OrcFileFormat import org.apache.spark.sql.types._ + /** * Legacy catalog for interacting with the Hive metastore. * From 2568193f91b9ae129c19a67bfd514065215840ac Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Fri, 1 Jul 2016 18:34:30 +0800 Subject: [PATCH 28/45] add ut --- .../MetadataOnlyOptimizerSuite.scala | 87 +++++++++++++++++++ 1 file changed, 87 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala new file mode 100644 index 000000000000..045a536e9cb0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class MetadataOnlyOptimizerSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) + .toDF("id", "data", "partId", "part") + data.write.partitionBy("partId", "part").mode("append").saveAsTable("srcpart_15752") + } + + private def checkWithMetadataOnly(df: DataFrame): Unit = { + val localRelations = df.queryExecution.optimizedPlan.collect { + case l @ LocalRelation(_, _) => l + } + assert(localRelations.size == 1) + } + + private def checkWithoutMetadataOnly(df: DataFrame): Unit = { + val localRelations = df.queryExecution.optimizedPlan.collect{ + case l @ LocalRelation(_, _) => l + } + assert(localRelations.size == 0) + } + + test("spark-15752 metadata only optimizer for partition table") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + checkWithMetadataOnly(sql("select part from srcpart_15752 where part = 0 group by part")) + checkWithMetadataOnly(sql("select max(part) from srcpart_15752")) + checkWithMetadataOnly(sql("select max(part) from srcpart_15752 where part = 0")) + checkWithMetadataOnly( + sql("select part, min(partId) from srcpart_15752 where part = 0 group by part")) + checkWithMetadataOnly( + sql("select max(x) from (select part + 1 as x from srcpart_15752 where part = 1) t")) + checkWithMetadataOnly(sql("select distinct part from srcpart_15752")) + checkWithMetadataOnly(sql("select distinct part, partId from srcpart_15752")) + checkWithMetadataOnly( + sql("select distinct x from (select part + 1 as x from srcpart_15752 where part = 0) t")) + + // Now donot support metadata only optimizer + checkWithoutMetadataOnly(sql("select part, max(id) from srcpart_15752 group by part")) + checkWithoutMetadataOnly(sql("select distinct part, id from srcpart_15752")) + checkWithoutMetadataOnly(sql("select part, sum(partId) from srcpart_15752 group by part")) + checkWithoutMetadataOnly( + sql("select part from srcpart_15752 where part = 1 group by rollup(part)")) + checkWithoutMetadataOnly( + sql("select part from (select part from srcpart_15752 where part = 0 union all " + + "select part from srcpart_15752 where part= 1)t group by part")) + } + } + + test("spark-15752 without metadata only optimizer for partition table") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + checkWithoutMetadataOnly(sql("select part from srcpart_15752 where part = 0 group by part")) + checkWithoutMetadataOnly(sql("select max(part) from srcpart_15752")) + checkWithoutMetadataOnly(sql("select max(part) from srcpart_15752 where part = 0")) + checkWithoutMetadataOnly( + sql("select max(x) from (select part + 1 as x from srcpart_15752 where part = 1) t")) + checkWithoutMetadataOnly(sql("select distinct part from srcpart_15752")) + checkWithoutMetadataOnly( + sql("select distinct x from (select part + 1 as x from srcpart_15752 where part = 1) t")) + } + } +} From 26a97f4df483fe324b76c4ee3c1ddec2f3830566 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Mon, 4 Jul 2016 19:20:42 +0800 Subject: [PATCH 29/45] address comments --- .../sql/execution/MetadataOnlyOptimizer.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 22 ++-- .../MetadataOnlyOptimizerSuite.scala | 8 ++ .../sql/hive/execution/SQLQuerySuite.scala | 112 +++++++++--------- 4 files changed, 80 insertions(+), 64 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala index ff500add2d9d..73b8efe8ce17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela * 1. aggregate expression is partition columns. * e.g. SELECT col FROM tbl GROUP BY col. * 2. aggregate function on partition columns with DISTINCT. - * e.g. SELECT count(DISTINCT col) FROM tbl GROUP BY col. + * e.g. SELECT count(DISTINCT col1) FROM tbl GROUP BY col2. * 3. aggregate function on partition columns which have same result w or w/o DISTINCT keyword. * e.g. SELECT Max(col2) FROM tbl GROUP BY col1. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 49ac49c5b2c1..bdd7241545f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2867,13 +2867,19 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("spark-15752 metadata only optimizer for datasource table") { - val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) - .toDF("id", "data", "partId", "part") - data.write.partitionBy("partId", "part").mode("append").saveAsTable("srcpart_15752") - checkAnswer(sql("select max(part) from srcpart_15752"), Row("odd")) - checkAnswer(sql("select max(part) from srcpart_15752 where partId = 0"), Row("even")) - checkAnswer(sql("select max(part) from (select part from srcpart_15752) t"), Row("odd")) - checkAnswer(sql("select distinct part from srcpart_15752"), Row("even") :: Row("odd") :: Nil) - checkAnswer(sql("select distinct part from srcpart_15752 where partId = 0"), Row("even")) + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + withTable("srcpart_15752") { + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) + .toDF("id", "data", "partId", "part") + data.write.partitionBy("partId", "part").mode("append").saveAsTable("srcpart_15752") + checkAnswer(sql("select max(part) from srcpart_15752"), Row("odd")) + checkAnswer(sql("select max(part) from srcpart_15752 where partId = 0"), Row("even")) + checkAnswer(sql("select max(part) from (select part from srcpart_15752) t"), Row("odd")) + checkAnswer( + sql("select distinct part from srcpart_15752"), + Row("even") :: Row("odd") :: Nil) + checkAnswer(sql("select distinct part from srcpart_15752 where partId = 0"), Row("even")) + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala index 045a536e9cb0..55dc4ad5da5f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala @@ -32,6 +32,14 @@ class MetadataOnlyOptimizerSuite extends QueryTest with SharedSQLContext { data.write.partitionBy("partId", "part").mode("append").saveAsTable("srcpart_15752") } + override protected def afterAll(): Unit = { + try { + sql("DROP TABLE IF EXISTS srcpart_15752") + } finally { + super.afterAll() + } + } + private def checkWithMetadataOnly(df: DataFrame): Unit = { val localRelations = df.queryExecution.optimizedPlan.collect { case l @ LocalRelation(_, _) => l 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 a7215faaca21..151a001562d8 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 @@ -1692,66 +1692,68 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("spark-15752 metadata only optimizer for hive table") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - val df = Seq((1, "2"), (3, "4")).toDF("key", "value") - df.createOrReplaceTempView("data_15752") - sql( - """ - |CREATE TABLE srcpart_15752 (key INT, value STRING) - |PARTITIONED BY (ds STRING, hr INT) STORED AS parquet - """.stripMargin) - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { + withTable("data_15752", "srcpart_15752", "srctext_15752") { + val df = Seq((1, "2"), (3, "4")).toDF("key", "value") + df.createOrReplaceTempView("data_15752") sql( - s""" - |INSERT OVERWRITE TABLE srcpart_15752 PARTITION (ds='$ds',hr='$hr') - |select key, value from data_15752 - """.stripMargin) - } - checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) - checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) - checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) - checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) - checkAnswer( - sql("select max(x) from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), - Row(13)) - checkAnswer(sql("select distinct hr from srcpart_15752"), Row(11) :: Row(12) :: Nil) - checkAnswer(sql("select distinct hr from srcpart_15752 where hr = 11"), Row(11)) - checkAnswer( - sql("select distinct x from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), - Row(13)) + """ + |CREATE TABLE srcpart_15752 (key INT, value STRING) + |PARTITIONED BY (ds STRING, hr INT) STORED AS parquet + """.stripMargin) + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { + sql( + s""" + |INSERT OVERWRITE TABLE srcpart_15752 PARTITION (ds='$ds',hr='$hr') + |select key, value from data_15752 + """.stripMargin) + } + checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) + checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) + checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) + checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) + checkAnswer( + sql("select max(x) from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + Row(13)) + checkAnswer(sql("select distinct hr from srcpart_15752"), Row(11) :: Row(12) :: Nil) + checkAnswer(sql("select distinct hr from srcpart_15752 where hr = 11"), Row(11)) + checkAnswer( + sql("select distinct x from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + Row(13)) - // Now donot support metadata only optimizer - checkAnswer( - sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), - Row(null) :: Row(12) :: Nil) - checkAnswer( - sql("select hr from (select hr from srcpart_15752 where hr = 11 union all " + - "select hr from srcpart_15752 where hr= 12)t group by hr"), - Row(11) :: Row(12) :: Nil) + // Now donot support metadata only optimizer + checkAnswer( + sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), + Row(null) :: Row(12) :: Nil) + checkAnswer( + sql("select hr from (select hr from srcpart_15752 where hr = 11 union all " + + "select hr from srcpart_15752 where hr= 12)t group by hr"), + Row(11) :: Row(12) :: Nil) - sql( - """ - |CREATE TABLE srctext_15752 (key INT, value STRING) - |PARTITIONED BY (ds STRING, hr INT) STORED AS textfile - """.stripMargin) - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { sql( - s""" - |INSERT OVERWRITE TABLE srctext_15752 PARTITION (ds='$ds',hr='$hr') - |select key, value from data_15752 - """.stripMargin) + """ + |CREATE TABLE srctext_15752 (key INT, value STRING) + |PARTITIONED BY (ds STRING, hr INT) STORED AS textfile + """.stripMargin) + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { + sql( + s""" + |INSERT OVERWRITE TABLE srctext_15752 PARTITION (ds='$ds',hr='$hr') + |select key, value from data_15752 + """.stripMargin) + } + checkAnswer(sql("select hr from srctext_15752 where hr = 11 group by hr"), Row(11)) + checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) + checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) + checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) + checkAnswer( + sql("select max(x) from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + Row(13)) + checkAnswer(sql("select distinct hr from srctext_15752"), Row(11) :: Row(12) :: Nil) + checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) + checkAnswer( + sql("select distinct x from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + Row(13)) } - checkAnswer(sql("select hr from srctext_15752 where hr = 11 group by hr"), Row(11)) - checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) - checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) - checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) - checkAnswer( - sql("select max(x) from (select hr + 1 as x from srctext_15752 where hr = 12) t"), - Row(13)) - checkAnswer(sql("select distinct hr from srctext_15752"), Row(11) :: Row(12) :: Nil) - checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) - checkAnswer( - sql("select distinct x from (select hr + 1 as x from srctext_15752 where hr = 12) t"), - Row(13)) } } } From 4297f9f7f32fc2ea59bba43569b17da94ed84fce Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 6 Jul 2016 11:14:09 +0800 Subject: [PATCH 30/45] update name --- .../{MetadataOnlyOptimizer.scala => OptimizeMetadataOnly.scala} | 2 +- .../scala/org/apache/spark/sql/execution/SparkOptimizer.scala | 2 +- ...OnlyOptimizerSuite.scala => OptimizeMetadataOnlySuite.scala} | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/{MetadataOnlyOptimizer.scala => OptimizeMetadataOnly.scala} (99%) rename sql/core/src/test/scala/org/apache/spark/sql/execution/{MetadataOnlyOptimizerSuite.scala => OptimizeMetadataOnlySuite.scala} (98%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnly.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnly.scala index 73b8efe8ce17..d17b5f4ba2db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnly.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRela * 3. aggregate function on partition columns which have same result w or w/o DISTINCT keyword. * e.g. SELECT Max(col2) FROM tbl GROUP BY col1. */ -case class MetadataOnlyOptimizer( +case class OptimizeMetadataOnly( catalog: SessionCatalog, conf: CatalystConf) extends Rule[LogicalPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index b78390fb5d86..26393c5f1839 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -30,7 +30,7 @@ class SparkOptimizer( extends Optimizer(catalog, conf) { override def batches: Seq[Batch] = super.batches :+ - Batch("Metadata Only Optimization", Once, MetadataOnlyOptimizer(catalog, conf)) :+ + Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnly(catalog, conf)) :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlySuite.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlySuite.scala index 55dc4ad5da5f..d3978dd498d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/MetadataOnlyOptimizerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlySuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -class MetadataOnlyOptimizerSuite extends QueryTest with SharedSQLContext { +class OptimizeMetadataOnlySuite extends QueryTest with SharedSQLContext { import testImplicits._ override def beforeAll(): Unit = { From 1a65aa7c57e3be349a8823fedc51365e7332981e Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 6 Jul 2016 20:14:36 +0800 Subject: [PATCH 31/45] address comments --- docs/sql-programming-guide.md | 11 ++ .../spark/sql/catalyst/CatalystConf.scala | 2 - ....scala => OptimizeMetadataOnlyQuery.scala} | 54 ++++--- .../spark/sql/execution/SparkOptimizer.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 5 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 22 +-- .../OptimizeMetadataOnlyQuerySuite.scala | 153 ++++++++++++++++++ .../execution/OptimizeMetadataOnlySuite.scala | 95 ----------- .../sql/hive/execution/SQLQuerySuite.scala | 75 +++++---- 9 files changed, 255 insertions(+), 164 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/{OptimizeMetadataOnly.scala => OptimizeMetadataOnlyQuery.scala} (71%) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlySuite.scala diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 68419e133159..addf25787219 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1476,6 +1476,17 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession

+ + spark.sql.optimizer.metadataOnly + true + +

+ When true, enable the metadata-only query optimization that use the table's metadata to + produce the partition columns instead of table scans. It applies when all the columns scanned + are partition columns and the query has an aggregate operator have distinct semantics. +

+ + ## JSON Datasets diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala index cabef8d29782..4df100c2a830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala @@ -28,7 +28,6 @@ trait CatalystConf { def orderByOrdinal: Boolean def groupByOrdinal: Boolean - def optimizerMetadataOnly: Boolean def optimizerMaxIterations: Int def optimizerInSetConversionThreshold: Int def maxCaseBranchesForCodegen: Int @@ -52,7 +51,6 @@ case class SimpleCatalystConf( caseSensitiveAnalysis: Boolean, orderByOrdinal: Boolean = true, groupByOrdinal: Boolean = true, - optimizerMetadataOnly: Boolean = true, optimizerMaxIterations: Int = 100, optimizerInSetConversionThreshold: Int = 10, maxCaseBranchesForCodegen: Int = 20, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnly.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala similarity index 71% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnly.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index d17b5f4ba2db..1bef83aead1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnly.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -25,21 +25,22 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.internal.SQLConf /** - * When scanning only partition columns, get results based on partition data without scanning files. - * It's used for operators that only need distinct values. Currently only [[Aggregate]] operator - * which satisfy the following conditions are supported: + * This rule optimizes the execution of queries that can be answered by looking only at + * partition-level metadata. This applies when all the columns scanned are partition columns, and + * the query has an aggregate operator that satisfies the following conditions: * 1. aggregate expression is partition columns. * e.g. SELECT col FROM tbl GROUP BY col. * 2. aggregate function on partition columns with DISTINCT. - * e.g. SELECT count(DISTINCT col1) FROM tbl GROUP BY col2. + * e.g. SELECT col1, count(DISTINCT col2) FROM tbl GROUP BY col1. * 3. aggregate function on partition columns which have same result w or w/o DISTINCT keyword. - * e.g. SELECT Max(col2) FROM tbl GROUP BY col1. + * e.g. SELECT col1, Max(col2) FROM tbl GROUP BY col1. */ -case class OptimizeMetadataOnly( +case class OptimizeMetadataOnlyQuery( catalog: SessionCatalog, - conf: CatalystConf) extends Rule[LogicalPlan] { + conf: SQLConf) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { if (!conf.optimizerMetadataOnly) { @@ -52,7 +53,7 @@ case class OptimizeMetadataOnly( val aggFunctions = aggExprs.flatMap(_.collect { case agg: AggregateExpression => agg }) - val isPartitionDataOnly = aggFunctions.isEmpty || aggFunctions.forall { agg => + val isPartitionDataOnly = aggFunctions.forall { agg => agg.isDistinct || (agg.aggregateFunction match { case _: Max => true case _: Min => true @@ -60,7 +61,7 @@ case class OptimizeMetadataOnly( }) } if (isPartitionDataOnly) { - a.withNewChildren(Seq(usePartitionData(child, relation))) + a.withNewChildren(Seq(replaceTableScanWithPartitionMetadata(child, relation))) } else { a } @@ -70,14 +71,16 @@ case class OptimizeMetadataOnly( } } - private def usePartitionData(child: LogicalPlan, relation: LogicalPlan): LogicalPlan = { + private def replaceTableScanWithPartitionMetadata( + child: LogicalPlan, + relation: LogicalPlan): LogicalPlan = { child transform { case plan if plan eq relation => relation match { case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) => val partColumns = fsRelation.partitionSchema.map(_.name.toLowerCase).toSet val partAttrs = l.output.filter(a => partColumns.contains(a.name.toLowerCase)) - val partitionData = fsRelation.location.listFiles(Nil) + val partitionData = fsRelation.location.listFiles(filters = Nil) LocalRelation(partAttrs, partitionData.map(_.values)) case relation: CatalogRelation => @@ -90,11 +93,19 @@ case class OptimizeMetadataOnly( } LocalRelation(partAttrs, partitionData) - case _ => throw new IllegalStateException() + case _ => + throw new IllegalStateException(s"The plan: $relation Cannot be replaced by " + + s"LocalRelation in the OptimizeMetadataOnlyQuery.") } } } + /** + * A pattern that finds scanned partition attributes and table relation all of whose columns + * scanned are partition columns. This applies when project or filter operators with + * deterministic expressions scan only partition columns. + * It returns scanned partition attributes and table relation plan, otherwise None. + */ object PartitionedRelation { def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) @@ -109,24 +120,15 @@ case class OptimizeMetadataOnly( Some(AttributeSet(partAttrs), relation) case p @ Project(projectList, child) if projectList.forall(_.deterministic) => - unapply(child).flatMap { - case (partAttrs, relation) => - if (p.references.subsetOf(partAttrs)) { - Some(p.outputSet, relation) - } else { - None - } + unapply(child).flatMap { case (partAttrs, relation) => + if (p.references.subsetOf(partAttrs)) Some(p.outputSet, relation) else None } case f @ Filter(condition, child) if condition.deterministic => - unapply(child).flatMap { - case (partAttrs, relation) => - if (f.references.subsetOf(partAttrs)) { - Some(f.outputSet, relation) - } else { - None - } + unapply(child).flatMap { case (partAttrs, relation) => + if (f.references.subsetOf(partAttrs)) Some(f.outputSet, relation) else None } + case _ => None } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 26393c5f1839..8b762b5d6c5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -30,7 +30,7 @@ class SparkOptimizer( extends Optimizer(catalog, conf) { override def batches: Seq[Batch] = super.batches :+ - Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnly(catalog, conf)) :+ + Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog, conf)) :+ Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9ed640d6e6a4..7190b135a2f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -259,7 +259,10 @@ object SQLConf { .createWithDefault(false) val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") - .doc("When true, enable the metadata-only query optimization.") + .doc("When true, enable the metadata-only query optimization that use the table's metadata " + + "to produce the partition columns instead of table scans. It applies when all the columns " + + "scanned are partition columns and the query has an aggregate operator have distinct " + + "semantics.") .booleanConf .createWithDefault(true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index bdd7241545f6..4471a197a9a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2866,19 +2866,23 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(s"$expected") :: Nil) } - test("spark-15752 metadata only optimizer for datasource table") { + test("spark-15752 optimize metadata only query for datasource table") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { withTable("srcpart_15752") { val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) - .toDF("id", "data", "partId", "part") - data.write.partitionBy("partId", "part").mode("append").saveAsTable("srcpart_15752") - checkAnswer(sql("select max(part) from srcpart_15752"), Row("odd")) - checkAnswer(sql("select max(part) from srcpart_15752 where partId = 0"), Row("even")) - checkAnswer(sql("select max(part) from (select part from srcpart_15752) t"), Row("odd")) + .toDF("col1", "col2", "partcol1", "partcol2") + data.write.partitionBy("partcol1", "partcol2").mode("append").saveAsTable("srcpart_15752") + checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(1)) + checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 0"), Row(0)) checkAnswer( - sql("select distinct part from srcpart_15752"), - Row("even") :: Row("odd") :: Nil) - checkAnswer(sql("select distinct part from srcpart_15752 where partId = 0"), Row("even")) + sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), + Row(1)) + checkAnswer( + sql("select distinct partcol1 from srcpart_15752"), + Row(0) :: Row(1) :: Nil) + checkAnswer( + sql("select distinct partcol1 from srcpart_15752 where partcol1 = 0"), + Row(0)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala new file mode 100644 index 000000000000..566c49442074 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + override def beforeAll(): Unit = { + super.beforeAll() + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) + .toDF("col1", "col2", "partcol1", "partcol2") + data.write.partitionBy("partcol1", "partcol2").mode("append").saveAsTable("srcpart") + } + + override protected def afterAll(): Unit = { + try { + sql("DROP TABLE IF EXISTS srcpart") + } finally { + super.afterAll() + } + } + + private def assertMetadataOnlyQuery(df: DataFrame): Unit = { + val localRelations = df.queryExecution.optimizedPlan.collect { + case l @ LocalRelation(_, _) => l + } + assert(localRelations.size == 1) + } + + private def assertNotMetadataOnlyQuery(df: DataFrame): Unit = { + val localRelations = df.queryExecution.optimizedPlan.collect { + case l @ LocalRelation(_, _) => l + } + assert(localRelations.size == 0) + } + + test("OptimizeMetadataOnlyQuery test: aggregate expression is partition columns") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + assertMetadataOnlyQuery(sql("select partcol1 from srcpart group by partcol1")) + assertMetadataOnlyQuery( + sql("select partcol2 from srcpart where partcol1 = 0 group by partcol2")) + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + assertNotMetadataOnlyQuery(sql("select partcol1 from srcpart group by partcol1")) + assertNotMetadataOnlyQuery( + sql("select partcol2 from srcpart where partcol1 = 0 group by partcol2")) + } + } + + test("OptimizeMetadataOnlyQuery test: distinct aggregate function on partition columns") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + assertMetadataOnlyQuery( + sql("SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1")) + assertMetadataOnlyQuery( + sql("SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 " + + "group by partcol1")) + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + assertNotMetadataOnlyQuery( + sql("SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1")) + assertNotMetadataOnlyQuery( + sql("SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 " + + "group by partcol1")) + } + } + + test("OptimizeMetadataOnlyQuery test: distinct on partition columns") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + assertMetadataOnlyQuery(sql("select distinct partcol1, partcol2 from srcpart")) + assertMetadataOnlyQuery( + sql("select distinct c1 from (select partcol1 + 1 as c1 from srcpart " + + "where partcol1 = 0) t")) + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + assertNotMetadataOnlyQuery(sql("select distinct partcol1, partcol2 from srcpart")) + assertNotMetadataOnlyQuery( + sql("select distinct c1 from (select partcol1 + 1 as c1 from srcpart " + + "where partcol1 = 0) t")) + } + } + + test("OptimizeMetadataOnlyQuery test: aggregate function on partition columns " + + "which have same result w or w/o DISTINCT keyword.") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + assertMetadataOnlyQuery(sql("select max(partcol1) from srcpart")) + assertMetadataOnlyQuery(sql("select max(partcol1) from srcpart where partcol1 = 0")) + assertMetadataOnlyQuery( + sql("select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2")) + assertMetadataOnlyQuery( + sql("select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + assertNotMetadataOnlyQuery(sql("select max(partcol1) from srcpart")) + assertNotMetadataOnlyQuery(sql("select max(partcol1) from srcpart where partcol1 = 0")) + assertNotMetadataOnlyQuery( + sql("select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2")) + assertNotMetadataOnlyQuery( + sql("select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) + } + } + + test("OptimizeMetadataOnlyQuery test: unsupported for non-partition columns") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + assertNotMetadataOnlyQuery(sql("select col1 from srcpart group by col1")) + assertNotMetadataOnlyQuery( + sql("select partcol1, max(col1) from srcpart group by partcol1")) + assertNotMetadataOnlyQuery( + sql("select partcol1, count(distinct col1) from srcpart group by partcol1")) + assertNotMetadataOnlyQuery(sql("select distinct partcol1, col1 from srcpart")) + } + } + + test("OptimizeMetadataOnlyQuery test: unsupported for non-distinct aggregate function on " + + "partition columns") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + assertNotMetadataOnlyQuery( + sql("select partcol1, sum(partcol2) from srcpart group by partcol1")) + assertNotMetadataOnlyQuery( + sql("select partcol1, count(partcol2) from srcpart group by partcol1")) + } + } + + test("OptimizeMetadataOnlyQuery test: unsupported for GroupingSet/Union operator") { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + assertNotMetadataOnlyQuery( + sql("select partcol1, max(partcol2) from srcpart where partcol1 = 0 " + + "group by rollup (partcol1)")) + assertNotMetadataOnlyQuery( + sql("select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + + "select partcol2 from srcpart where partcol1 = 1) t group by partcol2")) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlySuite.scala deleted file mode 100644 index d3978dd498d7..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlySuite.scala +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution - -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext - -class OptimizeMetadataOnlySuite extends QueryTest with SharedSQLContext { - import testImplicits._ - - override def beforeAll(): Unit = { - super.beforeAll() - val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) - .toDF("id", "data", "partId", "part") - data.write.partitionBy("partId", "part").mode("append").saveAsTable("srcpart_15752") - } - - override protected def afterAll(): Unit = { - try { - sql("DROP TABLE IF EXISTS srcpart_15752") - } finally { - super.afterAll() - } - } - - private def checkWithMetadataOnly(df: DataFrame): Unit = { - val localRelations = df.queryExecution.optimizedPlan.collect { - case l @ LocalRelation(_, _) => l - } - assert(localRelations.size == 1) - } - - private def checkWithoutMetadataOnly(df: DataFrame): Unit = { - val localRelations = df.queryExecution.optimizedPlan.collect{ - case l @ LocalRelation(_, _) => l - } - assert(localRelations.size == 0) - } - - test("spark-15752 metadata only optimizer for partition table") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - checkWithMetadataOnly(sql("select part from srcpart_15752 where part = 0 group by part")) - checkWithMetadataOnly(sql("select max(part) from srcpart_15752")) - checkWithMetadataOnly(sql("select max(part) from srcpart_15752 where part = 0")) - checkWithMetadataOnly( - sql("select part, min(partId) from srcpart_15752 where part = 0 group by part")) - checkWithMetadataOnly( - sql("select max(x) from (select part + 1 as x from srcpart_15752 where part = 1) t")) - checkWithMetadataOnly(sql("select distinct part from srcpart_15752")) - checkWithMetadataOnly(sql("select distinct part, partId from srcpart_15752")) - checkWithMetadataOnly( - sql("select distinct x from (select part + 1 as x from srcpart_15752 where part = 0) t")) - - // Now donot support metadata only optimizer - checkWithoutMetadataOnly(sql("select part, max(id) from srcpart_15752 group by part")) - checkWithoutMetadataOnly(sql("select distinct part, id from srcpart_15752")) - checkWithoutMetadataOnly(sql("select part, sum(partId) from srcpart_15752 group by part")) - checkWithoutMetadataOnly( - sql("select part from srcpart_15752 where part = 1 group by rollup(part)")) - checkWithoutMetadataOnly( - sql("select part from (select part from srcpart_15752 where part = 0 union all " + - "select part from srcpart_15752 where part= 1)t group by part")) - } - } - - test("spark-15752 without metadata only optimizer for partition table") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { - checkWithoutMetadataOnly(sql("select part from srcpart_15752 where part = 0 group by part")) - checkWithoutMetadataOnly(sql("select max(part) from srcpart_15752")) - checkWithoutMetadataOnly(sql("select max(part) from srcpart_15752 where part = 0")) - checkWithoutMetadataOnly( - sql("select max(x) from (select part + 1 as x from srcpart_15752 where part = 1) t")) - checkWithoutMetadataOnly(sql("select distinct part from srcpart_15752")) - checkWithoutMetadataOnly( - sql("select distinct x from (select part + 1 as x from srcpart_15752 where part = 1) t")) - } - } -} 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 151a001562d8..3f62afad7382 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 @@ -1690,68 +1690,83 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("spark-15752 metadata only optimizer for hive table") { + test("spark-15752 optimize metadata only query for hive table") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { withTable("data_15752", "srcpart_15752", "srctext_15752") { val df = Seq((1, "2"), (3, "4")).toDF("key", "value") df.createOrReplaceTempView("data_15752") sql( """ - |CREATE TABLE srcpart_15752 (key INT, value STRING) - |PARTITIONED BY (ds STRING, hr INT) STORED AS parquet + |CREATE TABLE srcpart_15752 (col1 INT, col2 STRING) + |PARTITIONED BY (partcol1 INT, partcol2 STRING) STORED AS parquet """.stripMargin) - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { + for (partcol1 <- Seq(11, 12); partcol2 <- Seq("a", "b")) { sql( s""" - |INSERT OVERWRITE TABLE srcpart_15752 PARTITION (ds='$ds',hr='$hr') + |INSERT OVERWRITE TABLE srcpart_15752 + |PARTITION (partcol1='$partcol1', partcol2='$partcol2') |select key, value from data_15752 """.stripMargin) } - checkAnswer(sql("select hr from srcpart_15752 where hr = 11 group by hr"), Row(11)) - checkAnswer(sql("select max(hr) from srcpart_15752"), Row(12)) - checkAnswer(sql("select max(hr) from srcpart_15752 where hr = 11"), Row(11)) - checkAnswer(sql("select max(hr) from (select hr from srcpart_15752) t"), Row(12)) checkAnswer( - sql("select max(x) from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + sql("select partcol1 from srcpart_15752 where partcol1 = 11 group by partcol1"), + Row(11)) + checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(12)) + checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 11"), Row(11)) + checkAnswer( + sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), + Row(12)) + checkAnswer( + sql("select max(col) from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 12) t"), Row(13)) - checkAnswer(sql("select distinct hr from srcpart_15752"), Row(11) :: Row(12) :: Nil) - checkAnswer(sql("select distinct hr from srcpart_15752 where hr = 11"), Row(11)) + checkAnswer(sql("select distinct partcol1 from srcpart_15752"), Row(11) :: Row(12) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 11"), Row(11)) checkAnswer( - sql("select distinct x from (select hr + 1 as x from srcpart_15752 where hr = 12) t"), + sql("select distinct col from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 12) t"), Row(13)) // Now donot support metadata only optimizer checkAnswer( - sql("select hr from srcpart_15752 where hr = 12 group by rollup(hr)"), - Row(null) :: Row(12) :: Nil) + sql("select partcol1, max(partcol2) from srcpart_15752 where partcol1 = 12 " + + "group by rollup(partcol1)"), + Row(null, "b") :: Row(12, "b") :: Nil) checkAnswer( - sql("select hr from (select hr from srcpart_15752 where hr = 11 union all " + - "select hr from srcpart_15752 where hr= 12)t group by hr"), - Row(11) :: Row(12) :: Nil) + sql("select partcol2 from (select partcol2 from srcpart_15752 where partcol1 = 11 " + + "union all select partcol2 from srcpart_15752 where partcol1= 12)t group by partcol2"), + Row("a") :: Row("b") :: Nil) sql( """ - |CREATE TABLE srctext_15752 (key INT, value STRING) - |PARTITIONED BY (ds STRING, hr INT) STORED AS textfile + |CREATE TABLE srctext_15752 (col1 INT, col2 STRING) + |PARTITIONED BY (partcol1 INT, partcol2 STRING) STORED AS textfile """.stripMargin) - for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq(11, 12)) { + for (partcol1 <- Seq(11, 12); partcol2 <- Seq("a", "b")) { sql( s""" - |INSERT OVERWRITE TABLE srctext_15752 PARTITION (ds='$ds',hr='$hr') + |INSERT OVERWRITE TABLE srctext_15752 + |PARTITION (partcol1='$partcol1', partcol2='$partcol2') |select key, value from data_15752 """.stripMargin) } - checkAnswer(sql("select hr from srctext_15752 where hr = 11 group by hr"), Row(11)) - checkAnswer(sql("select max(hr) from srctext_15752"), Row(12)) - checkAnswer(sql("select max(hr) from srctext_15752 where hr = 11"), Row(11)) - checkAnswer(sql("select max(hr) from (select hr from srctext_15752) t"), Row(12)) checkAnswer( - sql("select max(x) from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + sql("select partcol1 from srctext_15752 where partcol1 = 11 group by partcol1"), + Row(11)) + checkAnswer(sql("select max(partcol1) from srctext_15752"), Row(12)) + checkAnswer(sql("select max(partcol1) from srctext_15752 where partcol1 = 11"), Row(11)) + checkAnswer( + sql("select max(partcol1) from (select partcol1 from srctext_15752) t"), + Row(12)) + checkAnswer( + sql("select max(col) from (select partcol1 + 1 as col from srctext_15752 " + + "where partcol1 = 12) t"), Row(13)) - checkAnswer(sql("select distinct hr from srctext_15752"), Row(11) :: Row(12) :: Nil) - checkAnswer(sql("select distinct hr from srctext_15752 where hr = 11"), Row(11)) + checkAnswer(sql("select distinct partcol1 from srctext_15752"), Row(11) :: Row(12) :: Nil) + checkAnswer(sql("select distinct partcol1 from srctext_15752 where partcol1 = 11"), Row(11)) checkAnswer( - sql("select distinct x from (select hr + 1 as x from srctext_15752 where hr = 12) t"), + sql("select distinct col from (select partcol1 + 1 as col from srctext_15752 " + + "where partcol1 = 12) t"), Row(13)) } } From d5e0df4acf3ce6a285b030c4cba42426e146024b Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 6 Jul 2016 20:21:13 +0800 Subject: [PATCH 32/45] update --- docs/sql-programming-guide.md | 2 +- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index addf25787219..b4b6e100f051 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1483,7 +1483,7 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession

When true, enable the metadata-only query optimization that use the table's metadata to produce the partition columns instead of table scans. It applies when all the columns scanned - are partition columns and the query has an aggregate operator have distinct semantics. + are partition columns and the query has an aggregate operator that has distinct semantics.

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 7190b135a2f2..91fa1c60dac1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -261,7 +261,7 @@ object SQLConf { val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + - "scanned are partition columns and the query has an aggregate operator have distinct " + + "scanned are partition columns and the query has an aggregate operator that has distinct " + "semantics.") .booleanConf .createWithDefault(true) From 9d6dd76944eb42411850f8b721a05e263a7db30e Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 6 Jul 2016 20:24:09 +0800 Subject: [PATCH 33/45] update2 --- docs/sql-programming-guide.md | 3 ++- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index b4b6e100f051..7096fd31a3fe 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1483,7 +1483,8 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession

When true, enable the metadata-only query optimization that use the table's metadata to produce the partition columns instead of table scans. It applies when all the columns scanned - are partition columns and the query has an aggregate operator that has distinct semantics. + are partition columns and the query has an aggregate operator that satisfies distinct + semantics.

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 91fa1c60dac1..9aa694f8e8fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -261,8 +261,8 @@ object SQLConf { val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + - "scanned are partition columns and the query has an aggregate operator that has distinct " + - "semantics.") + "scanned are partition columns and the query has an aggregate operator that satisfies " + + "distinct semantics.") .booleanConf .createWithDefault(true) From 9cb01d810bc993cae3c98962317026c1e5295ed2 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Wed, 6 Jul 2016 21:06:36 +0800 Subject: [PATCH 34/45] update --- .../spark/sql/execution/OptimizeMetadataOnlyQuery.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 1bef83aead1b..9ee0a96d7a3c 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 @@ -53,14 +53,14 @@ case class OptimizeMetadataOnlyQuery( val aggFunctions = aggExprs.flatMap(_.collect { case agg: AggregateExpression => agg }) - val isPartitionDataOnly = aggFunctions.forall { agg => + val hasAllDistinctAgg = aggFunctions.forall { agg => agg.isDistinct || (agg.aggregateFunction match { case _: Max => true case _: Min => true case _ => false }) } - if (isPartitionDataOnly) { + if (hasAllDistinctAgg) { a.withNewChildren(Seq(replaceTableScanWithPartitionMetadata(child, relation))) } else { a From 3e2687ded84965cecc5c0a251d0bc4f086812272 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 7 Jul 2016 01:56:22 +0800 Subject: [PATCH 35/45] doc improve --- .../execution/OptimizeMetadataOnlyQuery.scala | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) 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 9ee0a96d7a3c..bf16c3643da3 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 @@ -17,8 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{CatalystConf, InternalRow} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -49,18 +48,21 @@ case class OptimizeMetadataOnlyQuery( plan.transform { case a @ Aggregate(_, aggExprs, child @ PartitionedRelation(partAttrs, relation)) => + // We only apply this optimization when only partitioned attributes are scanned. if (a.references.subsetOf(partAttrs)) { val aggFunctions = aggExprs.flatMap(_.collect { case agg: AggregateExpression => agg }) - val hasAllDistinctAgg = aggFunctions.forall { agg => + val isAllDistinctAgg = aggFunctions.forall { agg => agg.isDistinct || (agg.aggregateFunction match { + // `Max` and `Min` are always distinct aggregate functions no matter they have + // DISTINCT keyword or not, as the result will be same. case _: Max => true case _: Min => true case _ => false }) } - if (hasAllDistinctAgg) { + if (isAllDistinctAgg) { a.withNewChildren(Seq(replaceTableScanWithPartitionMetadata(child, relation))) } else { a @@ -71,6 +73,10 @@ case class OptimizeMetadataOnlyQuery( } } + /** + * Transform the given plan, find its table scan nodes that matches the given relation, and then + * replace the table scan node with its corresponding partition values. + */ private def replaceTableScanWithPartitionMetadata( child: LogicalPlan, relation: LogicalPlan): LogicalPlan = { @@ -93,18 +99,18 @@ case class OptimizeMetadataOnlyQuery( } LocalRelation(partAttrs, partitionData) - case _ => - throw new IllegalStateException(s"The plan: $relation Cannot be replaced by " + - s"LocalRelation in the OptimizeMetadataOnlyQuery.") + case _ => throw new IllegalStateException(s"unrecognized table scan node: $relation") } } } /** - * A pattern that finds scanned partition attributes and table relation all of whose columns - * scanned are partition columns. This applies when project or filter operators with - * deterministic expressions scan only partition columns. - * It returns scanned partition attributes and table relation plan, otherwise None. + * A pattern that finds the partitioned table relation node inside the given plan, and returns a + * pair of the partition attributes and the table relation node. + * + * It keeps traversing down the given plan tree if there is a [[Project]] or [[Filter]] with + * deterministic expressions, and returns result after reaching the partitioned table relation + * node. */ object PartitionedRelation { def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { @@ -126,7 +132,7 @@ case class OptimizeMetadataOnlyQuery( case f @ Filter(condition, child) if condition.deterministic => unapply(child).flatMap { case (partAttrs, relation) => - if (f.references.subsetOf(partAttrs)) Some(f.outputSet, relation) else None + if (f.references.subsetOf(partAttrs)) Some(partAttrs, relation) else None } case _ => None From 2b4faf321302adcb60f366dede0b75f8bbe368b9 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 7 Jul 2016 08:27:20 +0800 Subject: [PATCH 36/45] update --- .../spark/sql/execution/OptimizeMetadataOnlyQuery.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 bf16c3643da3..b5c32d022c31 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 @@ -99,7 +99,9 @@ case class OptimizeMetadataOnlyQuery( } LocalRelation(partAttrs, partitionData) - case _ => throw new IllegalStateException(s"unrecognized table scan node: $relation") + case _ => + throw new IllegalStateException(s"unrecognized table scan node: $relation, " + + s"please turn off ${SQLConf.OPTIMIZER_METADATA_ONLY.key} and try again.") } } } From a894bb7063ab8d37a99b6466e8fcda8e734867fe Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 7 Jul 2016 10:35:12 +0800 Subject: [PATCH 37/45] delete cases --- .../spark/sql/hive/execution/SQLQuerySuite.scala | 10 ---------- 1 file changed, 10 deletions(-) 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 3f62afad7382..7eac19e9f45a 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 @@ -1727,16 +1727,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { "where partcol1 = 12) t"), Row(13)) - // Now donot support metadata only optimizer - checkAnswer( - sql("select partcol1, max(partcol2) from srcpart_15752 where partcol1 = 12 " + - "group by rollup(partcol1)"), - Row(null, "b") :: Row(12, "b") :: Nil) - checkAnswer( - sql("select partcol2 from (select partcol2 from srcpart_15752 where partcol1 = 11 " + - "union all select partcol2 from srcpart_15752 where partcol1= 12)t group by partcol2"), - Row("a") :: Row("b") :: Nil) - sql( """ |CREATE TABLE srctext_15752 (col1 INT, col2 STRING) From 85b695bd2f259b9b42f9405f3fd109fe75a7a93a Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 7 Jul 2016 16:09:42 +0800 Subject: [PATCH 38/45] update ut --- .../org/apache/spark/sql/SQLQuerySuite.scala | 33 +++++--- .../sql/hive/execution/SQLQuerySuite.scala | 75 ++++++++++++------- 2 files changed, 70 insertions(+), 38 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4471a197a9a8..4e6d4b7805da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -2866,23 +2866,38 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(s"$expected") :: Nil) } - test("spark-15752 optimize metadata only query for datasource table") { + test("SPARK-15752 optimize metadata only query for datasource table") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { withTable("srcpart_15752") { - val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "even" else "odd")) + val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) "a" else "b")) .toDF("col1", "col2", "partcol1", "partcol2") data.write.partitionBy("partcol1", "partcol2").mode("append").saveAsTable("srcpart_15752") - checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(1)) - checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 0"), Row(0)) checkAnswer( - sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), + sql("select partcol1 from srcpart_15752 group by partcol1"), + Row(0) :: Row(1) :: Nil) + checkAnswer( + sql("select partcol1 from srcpart_15752 where partcol1 = 1 group by partcol1"), Row(1)) checkAnswer( - sql("select distinct partcol1 from srcpart_15752"), - Row(0) :: Row(1) :: Nil) + sql("select partcol1, count(distinct partcol2) from srcpart_15752 group by partcol1"), + Row(0, 1) :: Row(1, 1) :: Nil) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srcpart_15752 where partcol1 = 1 " + + "group by partcol1"), + Row(1, 1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752"), Row(0) :: Row(1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer( + sql("select distinct col from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 1) t"), + Row(2)) + checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(1)) + checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), Row(1)) checkAnswer( - sql("select distinct partcol1 from srcpart_15752 where partcol1 = 0"), - Row(0)) + sql("select max(col) from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 1) t"), + Row(2)) } } } 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 7eac19e9f45a..a43f0d0d7e97 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 @@ -1690,7 +1690,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("spark-15752 optimize metadata only query for hive table") { + test("SPARK-15752 optimize metadata only query for hive table") { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { withTable("data_15752", "srcpart_15752", "srctext_15752") { val df = Seq((1, "2"), (3, "4")).toDF("key", "value") @@ -1700,7 +1700,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |CREATE TABLE srcpart_15752 (col1 INT, col2 STRING) |PARTITIONED BY (partcol1 INT, partcol2 STRING) STORED AS parquet """.stripMargin) - for (partcol1 <- Seq(11, 12); partcol2 <- Seq("a", "b")) { + for (partcol1 <- Seq(0, 1); partcol2 <- Seq("a", "b")) { sql( s""" |INSERT OVERWRITE TABLE srcpart_15752 @@ -1709,30 +1709,39 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) } checkAnswer( - sql("select partcol1 from srcpart_15752 where partcol1 = 11 group by partcol1"), - Row(11)) - checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(12)) - checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 11"), Row(11)) + sql("select partcol1 from srcpart_15752 group by partcol1"), + Row(0) :: Row(1) :: Nil) checkAnswer( - sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), - Row(12)) + sql("select partcol1 from srcpart_15752 where partcol1 = 1 group by partcol1"), + Row(1)) checkAnswer( - sql("select max(col) from (select partcol1 + 1 as col from srcpart_15752 " + - "where partcol1 = 12) t"), - Row(13)) - checkAnswer(sql("select distinct partcol1 from srcpart_15752"), Row(11) :: Row(12) :: Nil) - checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 11"), Row(11)) + sql("select partcol1, count(distinct partcol2) from srcpart_15752 group by partcol1"), + Row(0, 2) :: Row(1, 2) :: Nil) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srcpart_15752 where partcol1 = 1 " + + "group by partcol1"), + Row(1, 2) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752"), Row(0) :: Row(1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 1"), Row(1)) checkAnswer( sql("select distinct col from (select partcol1 + 1 as col from srcpart_15752 " + - "where partcol1 = 12) t"), - Row(13)) + "where partcol1 = 1) t"), + Row(2)) + checkAnswer(sql("select distinct partcol1 from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from srcpart_15752"), Row(1)) + checkAnswer(sql("select max(partcol1) from srcpart_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from (select partcol1 from srcpart_15752) t"), Row(1)) + checkAnswer( + sql("select max(col) from (select partcol1 + 1 as col from srcpart_15752 " + + "where partcol1 = 1) t"), + Row(2)) sql( """ |CREATE TABLE srctext_15752 (col1 INT, col2 STRING) |PARTITIONED BY (partcol1 INT, partcol2 STRING) STORED AS textfile """.stripMargin) - for (partcol1 <- Seq(11, 12); partcol2 <- Seq("a", "b")) { + for (partcol1 <- Seq(0, 1); partcol2 <- Seq("a", "b")) { sql( s""" |INSERT OVERWRITE TABLE srctext_15752 @@ -1741,23 +1750,31 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { """.stripMargin) } checkAnswer( - sql("select partcol1 from srctext_15752 where partcol1 = 11 group by partcol1"), - Row(11)) - checkAnswer(sql("select max(partcol1) from srctext_15752"), Row(12)) - checkAnswer(sql("select max(partcol1) from srctext_15752 where partcol1 = 11"), Row(11)) + sql("select partcol1 from srctext_15752 group by partcol1"), + Row(0) :: Row(1) :: Nil) checkAnswer( - sql("select max(partcol1) from (select partcol1 from srctext_15752) t"), - Row(12)) + sql("select partcol1 from srctext_15752 where partcol1 = 1 group by partcol1"), + Row(1)) checkAnswer( - sql("select max(col) from (select partcol1 + 1 as col from srctext_15752 " + - "where partcol1 = 12) t"), - Row(13)) - checkAnswer(sql("select distinct partcol1 from srctext_15752"), Row(11) :: Row(12) :: Nil) - checkAnswer(sql("select distinct partcol1 from srctext_15752 where partcol1 = 11"), Row(11)) + sql("select partcol1, count(distinct partcol2) from srctext_15752 group by partcol1"), + Row(0, 2) :: Row(1, 2) :: Nil) + checkAnswer( + sql("select partcol1, count(distinct partcol2) from srctext_15752 where partcol1 = 1 " + + "group by partcol1"), + Row(1, 2) :: Nil) + checkAnswer(sql("select distinct partcol1 from srctext_15752"), Row(0) :: Row(1) :: Nil) + checkAnswer(sql("select distinct partcol1 from srctext_15752 where partcol1 = 1"), Row(1)) checkAnswer( sql("select distinct col from (select partcol1 + 1 as col from srctext_15752 " + - "where partcol1 = 12) t"), - Row(13)) + "where partcol1 = 1) t"), + Row(2)) + checkAnswer(sql("select max(partcol1) from srctext_15752"), Row(1)) + checkAnswer(sql("select max(partcol1) from srctext_15752 where partcol1 = 1"), Row(1)) + checkAnswer(sql("select max(partcol1) from (select partcol1 from srctext_15752) t"), Row(1)) + checkAnswer( + sql("select max(col) from (select partcol1 + 1 as col from srctext_15752 " + + "where partcol1 = 1) t"), + Row(2)) } } } From 67211beb80c4d84fb70c6037cc53044f86f094d5 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Thu, 7 Jul 2016 16:17:23 +0800 Subject: [PATCH 39/45] Merge branch 'master' of https://github.com/apache/spark into metadata-only --- .../apache/spark/sql/internal/SQLConf.scala | 23 ------------------- 1 file changed, 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 9aa694f8e8fd..14a1680fafa3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -266,25 +266,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val NATIVE_VIEW = SQLConfigBuilder("spark.sql.nativeView") - .internal() - .doc("When true, CREATE VIEW will be handled by Spark SQL instead of Hive native commands. " + - "Note that this function is experimental and should ony be used when you are using " + - "non-hive-compatible tables written by Spark SQL. The SQL string used to create " + - "view should be fully qualified, i.e. use `tbl1`.`col1` instead of `*` whenever " + - "possible, or you may get wrong result.") - .booleanConf - .createWithDefault(true) - - val CANONICAL_NATIVE_VIEW = SQLConfigBuilder("spark.sql.nativeView.canonical") - .internal() - .doc("When this option and spark.sql.nativeView are both true, Spark SQL tries to handle " + - "CREATE VIEW statement using SQL query string generated from view definition logical " + - "plan. If the logical plan doesn't have a SQL representation, we fallback to the " + - "original native view implementation.") - .booleanConf - .createWithDefault(true) - val COLUMN_NAME_OF_CORRUPT_RECORD = SQLConfigBuilder("spark.sql.columnNameOfCorruptRecord") .doc("The name of internal column for storing raw/un-parsed JSON records that fail to parse.") .stringConf @@ -623,8 +604,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY) - def nativeView: Boolean = getConf(NATIVE_VIEW) - def wholeStageEnabled: Boolean = getConf(WHOLESTAGE_CODEGEN_ENABLED) def wholeStageMaxNumFields: Int = getConf(WHOLESTAGE_MAX_NUM_FIELDS) @@ -635,8 +614,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED) - def canonicalView: Boolean = getConf(CANONICAL_NATIVE_VIEW) - def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE) def subexpressionEliminationEnabled: Boolean = From 501f93bde2c68b1c72ef9718a74edd2525551bc0 Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Mon, 11 Jul 2016 20:54:54 +0800 Subject: [PATCH 40/45] address commetns --- .../execution/OptimizeMetadataOnlyQuery.scala | 6 +- .../OptimizeMetadataOnlyQuerySuite.scala | 123 ++++++------------ 2 files changed, 47 insertions(+), 82 deletions(-) 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 b5c32d022c31..912d274ce02f 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 @@ -55,10 +55,12 @@ case class OptimizeMetadataOnlyQuery( }) val isAllDistinctAgg = aggFunctions.forall { agg => agg.isDistinct || (agg.aggregateFunction match { - // `Max` and `Min` are always distinct aggregate functions no matter they have - // DISTINCT keyword or not, as the result will be same. + // `Max`, `Min`, `First` and `Last` are always distinct aggregate functions no matter + // they have DISTINCT keyword or not, as the result will be same. case _: Max => true case _: Min => true + case _: First => true + case _: Last => true case _ => false }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index 566c49442074..7b690366f792 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -41,113 +41,76 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { } private def assertMetadataOnlyQuery(df: DataFrame): Unit = { - val localRelations = df.queryExecution.optimizedPlan.collect { - case l @ LocalRelation(_, _) => l + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + val localRelations = df.queryExecution.optimizedPlan.collect { + case l@LocalRelation(_, _) => l + } + assert(localRelations.size == 1) } - assert(localRelations.size == 1) } private def assertNotMetadataOnlyQuery(df: DataFrame): Unit = { - val localRelations = df.queryExecution.optimizedPlan.collect { - case l @ LocalRelation(_, _) => l + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + val localRelations = df.queryExecution.optimizedPlan.collect { + case l@LocalRelation(_, _) => l + } + assert(localRelations.size == 0) } - assert(localRelations.size == 0) } test("OptimizeMetadataOnlyQuery test: aggregate expression is partition columns") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - assertMetadataOnlyQuery(sql("select partcol1 from srcpart group by partcol1")) - assertMetadataOnlyQuery( - sql("select partcol2 from srcpart where partcol1 = 0 group by partcol2")) - } - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { - assertNotMetadataOnlyQuery(sql("select partcol1 from srcpart group by partcol1")) - assertNotMetadataOnlyQuery( - sql("select partcol2 from srcpart where partcol1 = 0 group by partcol2")) - } + assertMetadataOnlyQuery(sql("select partcol1 from srcpart group by partcol1")) + assertMetadataOnlyQuery( + sql("select partcol2 from srcpart where partcol1 = 0 group by partcol2")) } test("OptimizeMetadataOnlyQuery test: distinct aggregate function on partition columns") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - assertMetadataOnlyQuery( - sql("SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1")) - assertMetadataOnlyQuery( - sql("SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 " + - "group by partcol1")) - } - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { - assertNotMetadataOnlyQuery( - sql("SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1")) - assertNotMetadataOnlyQuery( - sql("SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 " + - "group by partcol1")) - } + assertMetadataOnlyQuery( + sql("SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1")) + assertMetadataOnlyQuery( + sql("SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 " + + "group by partcol1")) } test("OptimizeMetadataOnlyQuery test: distinct on partition columns") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - assertMetadataOnlyQuery(sql("select distinct partcol1, partcol2 from srcpart")) - assertMetadataOnlyQuery( - sql("select distinct c1 from (select partcol1 + 1 as c1 from srcpart " + - "where partcol1 = 0) t")) - } - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { - assertNotMetadataOnlyQuery(sql("select distinct partcol1, partcol2 from srcpart")) - assertNotMetadataOnlyQuery( - sql("select distinct c1 from (select partcol1 + 1 as c1 from srcpart " + - "where partcol1 = 0) t")) - } + assertMetadataOnlyQuery(sql("select distinct partcol1, partcol2 from srcpart")) + assertMetadataOnlyQuery( + sql("select distinct c1 from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) } test("OptimizeMetadataOnlyQuery test: aggregate function on partition columns " + "which have same result w or w/o DISTINCT keyword.") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - assertMetadataOnlyQuery(sql("select max(partcol1) from srcpart")) - assertMetadataOnlyQuery(sql("select max(partcol1) from srcpart where partcol1 = 0")) - assertMetadataOnlyQuery( - sql("select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2")) - assertMetadataOnlyQuery( - sql("select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) - } - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { - assertNotMetadataOnlyQuery(sql("select max(partcol1) from srcpart")) - assertNotMetadataOnlyQuery(sql("select max(partcol1) from srcpart where partcol1 = 0")) - assertNotMetadataOnlyQuery( - sql("select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2")) - assertNotMetadataOnlyQuery( - sql("select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) - } + assertMetadataOnlyQuery(sql("select max(partcol1) from srcpart")) + assertMetadataOnlyQuery(sql("select min(partcol1) from srcpart where partcol1 = 0")) + assertMetadataOnlyQuery(sql("select first(partcol1) from srcpart")) + assertMetadataOnlyQuery(sql("select last(partcol1) from srcpart where partcol1 = 0")) + assertMetadataOnlyQuery( + sql("select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2")) + assertMetadataOnlyQuery( + sql("select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) } test("OptimizeMetadataOnlyQuery test: unsupported for non-partition columns") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - assertNotMetadataOnlyQuery(sql("select col1 from srcpart group by col1")) - assertNotMetadataOnlyQuery( - sql("select partcol1, max(col1) from srcpart group by partcol1")) - assertNotMetadataOnlyQuery( - sql("select partcol1, count(distinct col1) from srcpart group by partcol1")) - assertNotMetadataOnlyQuery(sql("select distinct partcol1, col1 from srcpart")) - } + assertNotMetadataOnlyQuery(sql("select col1 from srcpart group by col1")) + assertNotMetadataOnlyQuery(sql("select partcol1, max(col1) from srcpart group by partcol1")) + assertNotMetadataOnlyQuery( + sql("select partcol1, count(distinct col1) from srcpart group by partcol1")) + assertNotMetadataOnlyQuery(sql("select distinct partcol1, col1 from srcpart")) } test("OptimizeMetadataOnlyQuery test: unsupported for non-distinct aggregate function on " + "partition columns") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - assertNotMetadataOnlyQuery( - sql("select partcol1, sum(partcol2) from srcpart group by partcol1")) - assertNotMetadataOnlyQuery( - sql("select partcol1, count(partcol2) from srcpart group by partcol1")) - } + assertNotMetadataOnlyQuery(sql("select partcol1, sum(partcol2) from srcpart group by partcol1")) + assertNotMetadataOnlyQuery( + sql("select partcol1, count(partcol2) from srcpart group by partcol1")) } test("OptimizeMetadataOnlyQuery test: unsupported for GroupingSet/Union operator") { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - assertNotMetadataOnlyQuery( - sql("select partcol1, max(partcol2) from srcpart where partcol1 = 0 " + - "group by rollup (partcol1)")) - assertNotMetadataOnlyQuery( - sql("select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + - "select partcol2 from srcpart where partcol1 = 1) t group by partcol2")) - } + assertNotMetadataOnlyQuery( + sql("select partcol1, max(partcol2) from srcpart where partcol1 = 0 " + + "group by rollup (partcol1)")) + assertNotMetadataOnlyQuery( + sql("select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + + "select partcol2 from srcpart where partcol1 = 1) t group by partcol2")) } } From 8ee2a8c82a80d53647a6f3c391cb41c22a07a17f Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 12 Jul 2016 02:17:56 +0800 Subject: [PATCH 41/45] refactor --- .../execution/OptimizeMetadataOnlyQuery.scala | 26 ++-- .../OptimizeMetadataOnlyQuerySuite.scala | 120 +++++++++--------- 2 files changed, 80 insertions(+), 66 deletions(-) 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 912d274ce02f..320dd6d69f54 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 @@ -86,14 +86,14 @@ case class OptimizeMetadataOnlyQuery( case plan if plan eq relation => relation match { case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) => - val partColumns = fsRelation.partitionSchema.map(_.name.toLowerCase).toSet - val partAttrs = l.output.filter(a => partColumns.contains(a.name.toLowerCase)) + val partAttrs = PartitionedRelation.getPartitionAttrs( + fsRelation.partitionSchema.map(_.name), l) val partitionData = fsRelation.location.listFiles(filters = Nil) LocalRelation(partAttrs, partitionData.map(_.values)) case relation: CatalogRelation => - val partColumns = relation.catalogTable.partitionColumnNames.map(_.toLowerCase).toSet - val partAttrs = relation.output.filter(a => partColumns.contains(a.name.toLowerCase)) + val partAttrs = PartitionedRelation.getPartitionAttrs( + relation.catalogTable.partitionColumnNames, relation) val partitionData = catalog.listPartitions(relation.catalogTable.identifier).map { p => InternalRow.fromSeq(partAttrs.map { attr => Cast(Literal(p.spec(attr.name)), attr.dataType).eval() @@ -116,17 +116,16 @@ case class OptimizeMetadataOnlyQuery( * deterministic expressions, and returns result after reaching the partitioned table relation * node. */ - object PartitionedRelation { + private[execution] object PartitionedRelation { + def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) if fsRelation.partitionSchema.nonEmpty => - val partColumns = fsRelation.partitionSchema.map(_.name.toLowerCase).toSet - val partAttrs = l.output.filter(a => partColumns.contains(a.name.toLowerCase)) + val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) Some(AttributeSet(partAttrs), l) case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => - val partColumns = relation.catalogTable.partitionColumnNames.map(_.toLowerCase).toSet - val partAttrs = relation.output.filter(a => partColumns.contains(a.name.toLowerCase)) + val partAttrs = getPartitionAttrs(relation.catalogTable.partitionColumnNames, relation) Some(AttributeSet(partAttrs), relation) case p @ Project(projectList, child) if projectList.forall(_.deterministic) => @@ -141,5 +140,14 @@ case class OptimizeMetadataOnlyQuery( case _ => None } + + /** + * Returns the partition attributes of the table relation plan. + */ + def getPartitionAttrs(partitionColumnNames: Seq[String], relation: LogicalPlan) + : Seq[Attribute] = { + val partColumns = partitionColumnNames.map(_.toLowerCase).toSet + relation.output.filter(a => partColumns.contains(a.name.toLowerCase)) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index 7b690366f792..928588b90111 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -41,76 +41,82 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { } private def assertMetadataOnlyQuery(df: DataFrame): Unit = { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - val localRelations = df.queryExecution.optimizedPlan.collect { - case l@LocalRelation(_, _) => l - } - assert(localRelations.size == 1) + val localRelations = df.queryExecution.optimizedPlan.collect { + case l @ LocalRelation(_, _) => l } + assert(localRelations.size == 1) } private def assertNotMetadataOnlyQuery(df: DataFrame): Unit = { - withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { - val localRelations = df.queryExecution.optimizedPlan.collect { - case l@LocalRelation(_, _) => l - } - assert(localRelations.size == 0) + val localRelations = df.queryExecution.optimizedPlan.collect { + case l @ LocalRelation(_, _) => l } + assert(localRelations.size == 0) } - test("OptimizeMetadataOnlyQuery test: aggregate expression is partition columns") { - assertMetadataOnlyQuery(sql("select partcol1 from srcpart group by partcol1")) - assertMetadataOnlyQuery( - sql("select partcol2 from srcpart where partcol1 = 0 group by partcol2")) + private def testMetadataOnly(name: String, sqls: String*): Unit = { + test(name) { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + sqls.foreach { case q => assertMetadataOnlyQuery(sql(q)) } + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) } + } + } } - test("OptimizeMetadataOnlyQuery test: distinct aggregate function on partition columns") { - assertMetadataOnlyQuery( - sql("SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1")) - assertMetadataOnlyQuery( - sql("SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 " + - "group by partcol1")) + private def testUnspportedMetadataOnly(name: String, sqls: String*): Unit = { + test(name) { + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { + sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) } + } + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") { + sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) } + } + } } - test("OptimizeMetadataOnlyQuery test: distinct on partition columns") { - assertMetadataOnlyQuery(sql("select distinct partcol1, partcol2 from srcpart")) - assertMetadataOnlyQuery( - sql("select distinct c1 from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) - } + testMetadataOnly( + "OptimizeMetadataOnlyQuery test: aggregate expression is partition columns", + "select partcol1 from srcpart group by partcol1", + "select partcol2 from srcpart where partcol1 = 0 group by partcol2") - test("OptimizeMetadataOnlyQuery test: aggregate function on partition columns " + - "which have same result w or w/o DISTINCT keyword.") { - assertMetadataOnlyQuery(sql("select max(partcol1) from srcpart")) - assertMetadataOnlyQuery(sql("select min(partcol1) from srcpart where partcol1 = 0")) - assertMetadataOnlyQuery(sql("select first(partcol1) from srcpart")) - assertMetadataOnlyQuery(sql("select last(partcol1) from srcpart where partcol1 = 0")) - assertMetadataOnlyQuery( - sql("select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2")) - assertMetadataOnlyQuery( - sql("select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t")) - } + testMetadataOnly( + "OptimizeMetadataOnlyQuery test: distinct aggregate function on partition columns", + "SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1", + "SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 group by partcol1") - test("OptimizeMetadataOnlyQuery test: unsupported for non-partition columns") { - assertNotMetadataOnlyQuery(sql("select col1 from srcpart group by col1")) - assertNotMetadataOnlyQuery(sql("select partcol1, max(col1) from srcpart group by partcol1")) - assertNotMetadataOnlyQuery( - sql("select partcol1, count(distinct col1) from srcpart group by partcol1")) - assertNotMetadataOnlyQuery(sql("select distinct partcol1, col1 from srcpart")) - } + testMetadataOnly( + "OptimizeMetadataOnlyQuery test: distinct on partition columns", + "select distinct partcol1, partcol2 from srcpart", + "select distinct c1 from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") - test("OptimizeMetadataOnlyQuery test: unsupported for non-distinct aggregate function on " + - "partition columns") { - assertNotMetadataOnlyQuery(sql("select partcol1, sum(partcol2) from srcpart group by partcol1")) - assertNotMetadataOnlyQuery( - sql("select partcol1, count(partcol2) from srcpart group by partcol1")) - } + testMetadataOnly( + "OptimizeMetadataOnlyQuery test: aggregate function on partition columns which have same " + + "result w or w/o DISTINCT keyword.", + "select max(partcol1) from srcpart", + "select min(partcol1) from srcpart where partcol1 = 0", + "select first(partcol1) from srcpart", + "select last(partcol1) from srcpart where partcol1 = 0", + "select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2", + "select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") - test("OptimizeMetadataOnlyQuery test: unsupported for GroupingSet/Union operator") { - assertNotMetadataOnlyQuery( - sql("select partcol1, max(partcol2) from srcpart where partcol1 = 0 " + - "group by rollup (partcol1)")) - assertNotMetadataOnlyQuery( - sql("select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + - "select partcol2 from srcpart where partcol1 = 1) t group by partcol2")) - } + testUnspportedMetadataOnly( + "OptimizeMetadataOnlyQuery test: unsupported for non-partition columns", + "select col1 from srcpart group by col1", + "select partcol1, max(col1) from srcpart group by partcol1", + "select partcol1, count(distinct col1) from srcpart group by partcol1", + "select distinct partcol1, col1 from srcpart") + + testUnspportedMetadataOnly( + "OptimizeMetadataOnlyQuery test: unsupported for non-distinct aggregate function on " + + "partition columns", + "select partcol1, sum(partcol2) from srcpart group by partcol1", + "select partcol1, count(partcol2) from srcpart group by partcol1") + + testUnspportedMetadataOnly( + "OptimizeMetadataOnlyQuery test: unsupported for GroupingSet/Union operator", + "select partcol1, max(partcol2) from srcpart where partcol1 = 0 group by rollup (partcol1)", + "select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + + "select partcol2 from srcpart where partcol1 = 1) t group by partcol2") } From d888c85a1b95904408d436271a40d27095e2eb4d Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 12 Jul 2016 02:20:22 +0800 Subject: [PATCH 42/45] fix minor --- .../sql/execution/OptimizeMetadataOnlyQuerySuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index 928588b90111..433afc3a3993 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -65,7 +65,7 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { } } - private def testUnspportedMetadataOnly(name: String, sqls: String*): Unit = { + private def testNotMetadataOnly(name: String, sqls: String*): Unit = { test(name) { withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") { sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) } @@ -101,20 +101,20 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { "select partcol2, min(partcol1) from srcpart where partcol1 = 0 group by partcol2", "select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") - testUnspportedMetadataOnly( + testNotMetadataOnly( "OptimizeMetadataOnlyQuery test: unsupported for non-partition columns", "select col1 from srcpart group by col1", "select partcol1, max(col1) from srcpart group by partcol1", "select partcol1, count(distinct col1) from srcpart group by partcol1", "select distinct partcol1, col1 from srcpart") - testUnspportedMetadataOnly( + testNotMetadataOnly( "OptimizeMetadataOnlyQuery test: unsupported for non-distinct aggregate function on " + "partition columns", "select partcol1, sum(partcol2) from srcpart group by partcol1", "select partcol1, count(partcol2) from srcpart group by partcol1") - testUnspportedMetadataOnly( + testNotMetadataOnly( "OptimizeMetadataOnlyQuery test: unsupported for GroupingSet/Union operator", "select partcol1, max(partcol2) from srcpart where partcol1 = 0 group by rollup (partcol1)", "select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + From ff1650987b901825d3828de972fa4434466c951f Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 12 Jul 2016 10:51:00 +0800 Subject: [PATCH 43/45] update --- .../execution/OptimizeMetadataOnlyQuery.scala | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) 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 320dd6d69f54..ee34a8a60b80 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 @@ -75,6 +75,16 @@ case class OptimizeMetadataOnlyQuery( } } + /** + * Returns the partition attributes of the table relation plan. + */ + private def getPartitionAttrs( + partitionColumnNames: Seq[String], + relation: LogicalPlan): Seq[Attribute] = { + val partColumns = partitionColumnNames.map(_.toLowerCase).toSet + relation.output.filter(a => partColumns.contains(a.name.toLowerCase)) + } + /** * Transform the given plan, find its table scan nodes that matches the given relation, and then * replace the table scan node with its corresponding partition values. @@ -86,14 +96,12 @@ case class OptimizeMetadataOnlyQuery( case plan if plan eq relation => relation match { case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) => - val partAttrs = PartitionedRelation.getPartitionAttrs( - fsRelation.partitionSchema.map(_.name), l) + val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) val partitionData = fsRelation.location.listFiles(filters = Nil) LocalRelation(partAttrs, partitionData.map(_.values)) case relation: CatalogRelation => - val partAttrs = PartitionedRelation.getPartitionAttrs( - relation.catalogTable.partitionColumnNames, relation) + val partAttrs = getPartitionAttrs(relation.catalogTable.partitionColumnNames, relation) val partitionData = catalog.listPartitions(relation.catalogTable.identifier).map { p => InternalRow.fromSeq(partAttrs.map { attr => Cast(Literal(p.spec(attr.name)), attr.dataType).eval() @@ -116,17 +124,17 @@ case class OptimizeMetadataOnlyQuery( * deterministic expressions, and returns result after reaching the partitioned table relation * node. */ - private[execution] object PartitionedRelation { + object PartitionedRelation { def unapply(plan: LogicalPlan): Option[(AttributeSet, LogicalPlan)] = plan match { case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) if fsRelation.partitionSchema.nonEmpty => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) - Some(AttributeSet(partAttrs), l) + Some(partAttrs, l) case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => val partAttrs = getPartitionAttrs(relation.catalogTable.partitionColumnNames, relation) - Some(AttributeSet(partAttrs), relation) + Some(partAttrs, relation) case p @ Project(projectList, child) if projectList.forall(_.deterministic) => unapply(child).flatMap { case (partAttrs, relation) => @@ -144,10 +152,11 @@ case class OptimizeMetadataOnlyQuery( /** * Returns the partition attributes of the table relation plan. */ - def getPartitionAttrs(partitionColumnNames: Seq[String], relation: LogicalPlan) - : Seq[Attribute] = { + private def getPartitionAttrs( + partitionColumnNames: Seq[String], + relation: LogicalPlan): AttributeSet = { val partColumns = partitionColumnNames.map(_.toLowerCase).toSet - relation.output.filter(a => partColumns.contains(a.name.toLowerCase)) + AttributeSet(relation.output.filter(a => partColumns.contains(a.name.toLowerCase))) } } } From 358ad13f58562716d2f376818d9b45898616ba4a Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 12 Jul 2016 17:28:48 +0800 Subject: [PATCH 44/45] remove duplicate code --- .../sql/execution/OptimizeMetadataOnlyQuery.scala | 14 ++------------ .../OptimizeMetadataOnlyQuerySuite.scala | 15 +++++++-------- 2 files changed, 9 insertions(+), 20 deletions(-) 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 ee34a8a60b80..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 @@ -130,11 +130,11 @@ case class OptimizeMetadataOnlyQuery( case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) if fsRelation.partitionSchema.nonEmpty => val partAttrs = getPartitionAttrs(fsRelation.partitionSchema.map(_.name), l) - Some(partAttrs, l) + Some(AttributeSet(partAttrs), l) case relation: CatalogRelation if relation.catalogTable.partitionColumnNames.nonEmpty => val partAttrs = getPartitionAttrs(relation.catalogTable.partitionColumnNames, relation) - Some(partAttrs, relation) + Some(AttributeSet(partAttrs), relation) case p @ Project(projectList, child) if projectList.forall(_.deterministic) => unapply(child).flatMap { case (partAttrs, relation) => @@ -148,15 +148,5 @@ case class OptimizeMetadataOnlyQuery( case _ => None } - - /** - * Returns the partition attributes of the table relation plan. - */ - private def getPartitionAttrs( - partitionColumnNames: Seq[String], - relation: LogicalPlan): AttributeSet = { - val partColumns = partitionColumnNames.map(_.toLowerCase).toSet - AttributeSet(relation.output.filter(a => partColumns.contains(a.name.toLowerCase))) - } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index 433afc3a3993..ba02f15503eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -77,23 +77,22 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { } testMetadataOnly( - "OptimizeMetadataOnlyQuery test: aggregate expression is partition columns", + "aggregate expression is partition columns", "select partcol1 from srcpart group by partcol1", "select partcol2 from srcpart where partcol1 = 0 group by partcol2") testMetadataOnly( - "OptimizeMetadataOnlyQuery test: distinct aggregate function on partition columns", + "distinct aggregate function on partition columns", "SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1", "SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 group by partcol1") testMetadataOnly( - "OptimizeMetadataOnlyQuery test: distinct on partition columns", + "distinct on partition columns", "select distinct partcol1, partcol2 from srcpart", "select distinct c1 from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") testMetadataOnly( - "OptimizeMetadataOnlyQuery test: aggregate function on partition columns which have same " + - "result w or w/o DISTINCT keyword.", + "aggregate function on partition columns which have same result w or w/o DISTINCT keyword.", "select max(partcol1) from srcpart", "select min(partcol1) from srcpart where partcol1 = 0", "select first(partcol1) from srcpart", @@ -102,20 +101,20 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { "select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") testNotMetadataOnly( - "OptimizeMetadataOnlyQuery test: unsupported for non-partition columns", + "unsupported for non-partition columns", "select col1 from srcpart group by col1", "select partcol1, max(col1) from srcpart group by partcol1", "select partcol1, count(distinct col1) from srcpart group by partcol1", "select distinct partcol1, col1 from srcpart") testNotMetadataOnly( - "OptimizeMetadataOnlyQuery test: unsupported for non-distinct aggregate function on " + + "unsupported for non-distinct aggregate function on " + "partition columns", "select partcol1, sum(partcol2) from srcpart group by partcol1", "select partcol1, count(partcol2) from srcpart group by partcol1") testNotMetadataOnly( - "OptimizeMetadataOnlyQuery test: unsupported for GroupingSet/Union operator", + "unsupported for GroupingSet/Union operator", "select partcol1, max(partcol2) from srcpart where partcol1 = 0 group by rollup (partcol1)", "select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + "select partcol2 from srcpart where partcol1 = 1) t group by partcol2") From 030776ae49484c4e5db7f775344e5e40dff27e9a Mon Sep 17 00:00:00 2001 From: Lianhui Wang Date: Tue, 12 Jul 2016 17:47:04 +0800 Subject: [PATCH 45/45] fix minor --- .../OptimizeMetadataOnlyQuerySuite.scala | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index ba02f15503eb..58c310596ca6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -77,22 +77,22 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { } testMetadataOnly( - "aggregate expression is partition columns", + "Aggregate expression is partition columns", "select partcol1 from srcpart group by partcol1", "select partcol2 from srcpart where partcol1 = 0 group by partcol2") testMetadataOnly( - "distinct aggregate function on partition columns", + "Distinct aggregate function on partition columns", "SELECT partcol1, count(distinct partcol2) FROM srcpart group by partcol1", "SELECT partcol1, count(distinct partcol2) FROM srcpart where partcol1 = 0 group by partcol1") testMetadataOnly( - "distinct on partition columns", + "Distinct on partition columns", "select distinct partcol1, partcol2 from srcpart", "select distinct c1 from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") testMetadataOnly( - "aggregate function on partition columns which have same result w or w/o DISTINCT keyword.", + "Aggregate function on partition columns which have same result w or w/o DISTINCT keyword", "select max(partcol1) from srcpart", "select min(partcol1) from srcpart where partcol1 = 0", "select first(partcol1) from srcpart", @@ -101,20 +101,19 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { "select max(c1) from (select partcol1 + 1 as c1 from srcpart where partcol1 = 0) t") testNotMetadataOnly( - "unsupported for non-partition columns", + "Don't optimize metadata only query for non-partition columns", "select col1 from srcpart group by col1", "select partcol1, max(col1) from srcpart group by partcol1", "select partcol1, count(distinct col1) from srcpart group by partcol1", "select distinct partcol1, col1 from srcpart") testNotMetadataOnly( - "unsupported for non-distinct aggregate function on " + - "partition columns", + "Don't optimize metadata only query for non-distinct aggregate function on partition columns", "select partcol1, sum(partcol2) from srcpart group by partcol1", "select partcol1, count(partcol2) from srcpart group by partcol1") testNotMetadataOnly( - "unsupported for GroupingSet/Union operator", + "Don't optimize metadata only query for GroupingSet/Union operator", "select partcol1, max(partcol2) from srcpart where partcol1 = 0 group by rollup (partcol1)", "select partcol2 from (select partcol2 from srcpart where partcol1 = 0 union all " + "select partcol2 from srcpart where partcol1 = 1) t group by partcol2")