Skip to content
Closed
Show file tree
Hide file tree
Changes from 11 commits
Commits
Show all changes
52 commits
Select commit Hold shift + click to select a range
2ca2c38
init commit
lianhuiwang Jun 3, 2016
edea710
fix unit test
lianhuiwang Jun 3, 2016
8426522
Merge branch 'apache-master' into metadata-only
lianhuiwang Jun 3, 2016
153293e
fix unit test
lianhuiwang Jun 3, 2016
7dfb743
update
lianhuiwang Jun 24, 2016
68e6d6d
Revert "fix unit test"
lianhuiwang Jun 24, 2016
595ef36
Revert "fix unit test"
lianhuiwang Jun 24, 2016
7d7ece0
Merge branch 'apache-master' into metadata-only
lianhuiwang Jun 24, 2016
2e55a9d
Merge branch 'apache-master' into metadata-only
lianhuiwang Jun 24, 2016
b2b6eba
update
lianhuiwang Jun 24, 2016
c5a291e
Merge branch 'apache-master' into metadata-only
lianhuiwang Jun 24, 2016
6404c1f
update opt for core
lianhuiwang Jun 24, 2016
1bb5812
refactor
lianhuiwang Jun 24, 2016
7e3729e
add ut
lianhuiwang Jun 24, 2016
fbf5d61
fix ut
lianhuiwang Jun 24, 2016
3411fd6
fix project
lianhuiwang Jun 26, 2016
aefab7f
address comments
lianhuiwang Jun 27, 2016
c5ccdea
fix cube/rollup
lianhuiwang Jun 27, 2016
ae6cf9f
fix style
lianhuiwang Jun 27, 2016
159331b
refactor
lianhuiwang Jun 27, 2016
3a1438b
refactor
lianhuiwang Jun 28, 2016
c0a7d59
update
lianhuiwang Jun 29, 2016
a4045ca
add comments
lianhuiwang Jun 29, 2016
0a023e7
fix minor
lianhuiwang Jun 29, 2016
a9b38ab
rename
lianhuiwang Jun 29, 2016
a5ea995
update
lianhuiwang Jun 29, 2016
1bed08d
fix monir
lianhuiwang Jun 29, 2016
a22e962
refactor
lianhuiwang Jul 1, 2016
41fef2c
update
lianhuiwang Jul 1, 2016
bd53678
Merge branch 'apache-master' into metadata-only
lianhuiwang Jul 1, 2016
88f7308
update
lianhuiwang Jul 1, 2016
2568193
add ut
lianhuiwang Jul 1, 2016
26a97f4
address comments
lianhuiwang Jul 4, 2016
4297f9f
update name
lianhuiwang Jul 6, 2016
1a65aa7
address comments
lianhuiwang Jul 6, 2016
d5e0df4
update
lianhuiwang Jul 6, 2016
9d6dd76
update2
lianhuiwang Jul 6, 2016
9cb01d8
update
lianhuiwang Jul 6, 2016
3e2687d
doc improve
cloud-fan Jul 6, 2016
2b4faf3
update
cloud-fan Jul 7, 2016
88fd3bf
Merge pull request #2 from cloud-fan/metadata-only
lianhuiwang Jul 7, 2016
a894bb7
delete cases
lianhuiwang Jul 7, 2016
9546b40
Merge branch 'metadata-only' of https://github.com/lianhuiwang/spark …
lianhuiwang Jul 7, 2016
85b695b
update ut
lianhuiwang Jul 7, 2016
bcfe8e5
Merge branch 'master' of https://github.com/apache/spark into metadat…
lianhuiwang Jul 7, 2016
67211be
Merge branch 'master' of https://github.com/apache/spark into metadat…
lianhuiwang Jul 7, 2016
501f93b
address commetns
lianhuiwang Jul 11, 2016
8ee2a8c
refactor
lianhuiwang Jul 11, 2016
d888c85
fix minor
lianhuiwang Jul 11, 2016
ff16509
update
lianhuiwang Jul 12, 2016
358ad13
remove duplicate code
lianhuiwang Jul 12, 2016
030776a
fix minor
lianhuiwang Jul 12, 2016
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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.")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please update the doc to explain what metadata-only query means.

.booleanConf
.createWithDefault(false)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we turn it on by default?


val NATIVE_VIEW = SQLConfigBuilder("spark.sql.nativeView")
.internal()
.doc("When true, CREATE VIEW will be handled by Spark SQL instead of Hive native commands. " +
Expand Down Expand Up @@ -613,6 +618,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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,17 +27,18 @@ 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, ParquetOptions}
import org.apache.spark.sql.hive.orc.OrcFileFormat
import org.apache.spark.sql.types._


/**
* Legacy catalog for interacting with the Hive metastore.
*
Expand Down Expand Up @@ -457,6 +458,125 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
allowExisting)
}
}

/**
* 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 =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we add more comments to explain the condition to enable metadata optimization? e.g. the agg expression must only reference to partition columns, all distinct agg functions, Max and Min, etc.

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 =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

MetastoreRelation extends CatalogRelation, I think we can put this rule in sql core instead of hive module.

(Some(relation), Seq.empty[Expression])

case p @ Project(_, child) =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it safe to keep traversing the plan tree through Project?

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(_, _)) =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why do we handle Distinct specially? It will be rewritten into Aggregate and I think we should have a general rule to handle Aggregate

val (plan, filters) = findRelation(p)
if (plan.isDefined) {
convertToMetadataOnlyPlan(d, Some(p), filters, plan.get)
} else {
d
}
}
}
}

}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,6 +87,7 @@ 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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -73,6 +74,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.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1684,4 +1684,47 @@ 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")
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 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(
"""
|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
""".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))
}
}
}