Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Next Next commit
refactor
  • Loading branch information
lianhuiwang committed Jun 28, 2016
commit 3a1438ba41c94ed44dbc0dc43c2457509f5e4fcc
Original file line number Diff line number Diff line change
Expand Up @@ -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]
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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 {
Copy link
Contributor

Choose a reason for hiding this comment

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

We need to think about it more carefully, i.e. how can the partition information propagate up from table relation?
It's obvious that Filter can retain all partition information, but for others, it's not trivial to explain.

Since this PR definitely need more people to review, how about we only handle Filter for now and improve it later? Then it's easier for other people to review and get this PR in. Thanks!

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)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand All @@ -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))
Expand Down