Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
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
push down operators to data source before planning
  • Loading branch information
cloud-fan committed Oct 10, 2017
commit e8e8feeeb54ae3e4f79157edb8b4f69886036cd0
Original file line number Diff line number Diff line change
Expand Up @@ -40,4 +40,12 @@ public interface SupportsPushDownCatalystFilters {
* Pushes down filters, and returns unsupported filters.
*/
Expression[] pushCatalystFilters(Expression[] filters);

/**
* Returns the catalyst filters that are pushed in {@link #pushCatalystFilters(Expression[])}.
* It's possible that there is no filters in the query and
* {@link #pushCatalystFilters(Expression[])} is never called, empty array should be returned for
* this case.
*/
Expression[] pushedCatalystFilters();
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.sources.v2.reader;

import org.apache.spark.annotation.InterfaceStability;
import org.apache.spark.sql.catalyst.expressions.Expression;
Copy link
Member

Choose a reason for hiding this comment

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

Seems we don't use Expression here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good catch. I'll remove it in my following PRs.

import org.apache.spark.sql.sources.Filter;

/**
Expand All @@ -35,4 +36,11 @@ public interface SupportsPushDownFilters {
* Pushes down filters, and returns unsupported filters.
*/
Filter[] pushFilters(Filter[] filters);

/**
* Returns the filters that are pushed in {@link #pushFilters(Filter[])}.
* It's possible that there is no filters in the query and {@link #pushFilters(Filter[])}
* is never called, empty array should be returned for this case.
*/
Filter[] pushedFilters();
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ 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.datasources.PruneFileSourcePartitions
import org.apache.spark.sql.execution.datasources.v2.PushDownOperatorsToDataSource
import org.apache.spark.sql.execution.python.ExtractPythonUDFFromAggregate

class SparkOptimizer(
Expand All @@ -31,7 +32,8 @@ class SparkOptimizer(
override def batches: Seq[Batch] = (preOptimizationBatches ++ super.batches :+
Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+
Batch("Extract Python UDF from Aggregate", Once, ExtractPythonUDFFromAggregate) :+
Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions)) ++
Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+
Batch("Push down operators to data source scan", Once, PushDownOperatorsToDataSource)) ++
postHocOptimizationBatches :+
Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*)

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* 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.datasources.v2

import java.util.Objects

import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.sources.v2.reader._

/**
* A base class for data source reader holder and defines equals/hashCode methods.
Copy link
Member

@gatorsmile gatorsmile Oct 11, 2017

Choose a reason for hiding this comment

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

defines -> with customized

*/
trait DataSourceReaderHolder {
def fullOutput: Seq[AttributeReference]
def reader: DataSourceV2Reader

override def equals(other: Any): Boolean = other match {
case other: DataSourceV2Relation =>
val basicEquals = this.fullOutput == other.fullOutput &&
this.reader.getClass == other.reader.getClass &&
this.reader.readSchema() == other.reader.readSchema()

val samePushedFilters = (this.reader, other.reader) match {
case (l: SupportsPushDownCatalystFilters, r: SupportsPushDownCatalystFilters) =>
l.pushedCatalystFilters().toSeq == r.pushedCatalystFilters().toSeq
case (l: SupportsPushDownFilters, r: SupportsPushDownFilters) =>
l.pushedFilters().toSeq == r.pushedFilters().toSeq
Copy link
Member

@gatorsmile gatorsmile Oct 11, 2017

Choose a reason for hiding this comment

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

The evaluation order of these filters must be the same? If the orders are different, they are still the same, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good catch!

case _ => true
}

basicEquals && samePushedFilters

case _ => false
}

override def hashCode(): Int = {
val state = Seq(fullOutput, reader.getClass, reader.readSchema())
val filters: Any = reader match {
case s: SupportsPushDownCatalystFilters => s.pushedCatalystFilters().toSeq
case s: SupportsPushDownFilters => s.pushedFilters().toSeq
case _ => Nil
}
(state :+ filters).map(Objects.hashCode).foldLeft(0)((a, b) => 31 * a + b)
}

lazy val output: Seq[Attribute] = reader.readSchema().map(_.name).map { name =>
fullOutput.find(_.name == name).get
Copy link
Member

Choose a reason for hiding this comment

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

Shall we use resolver instead of string comparison?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

These names should already be normalized before reaching here.

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources.v2

import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
import org.apache.spark.sql.sources.v2.reader.{DataSourceV2Reader, SupportsReportStatistics}
import org.apache.spark.sql.sources.v2.reader._

case class DataSourceV2Relation(
output: Seq[AttributeReference],
reader: DataSourceV2Reader) extends LeafNode {
fullOutput: Seq[AttributeReference],
reader: DataSourceV2Reader) extends LeafNode with DataSourceReaderHolder {

override def computeStats(): Statistics = reader match {
case r: SupportsReportStatistics =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,20 +29,12 @@ import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.sources.v2.reader._
import org.apache.spark.sql.types.StructType

/**
* Physical plan node for scanning data from a data source.
*/
case class DataSourceV2ScanExec(
Copy link
Member

Choose a reason for hiding this comment

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

/**
 * Physical plan node for scanning data from a data source.
 */

fullOutput: Array[AttributeReference],
@transient reader: DataSourceV2Reader,
// TODO: these 3 parameters are only used to determine the equality of the scan node, however,
// the reader also have this information, and ideally we can just rely on the equality of the
// reader. The only concern is, the reader implementation is outside of Spark and we have no
// control.
readSchema: StructType,
@transient filters: ExpressionSet,
hashPartitionKeys: Seq[String]) extends LeafExecNode {

def output: Seq[Attribute] = readSchema.map(_.name).map { name =>
fullOutput.find(_.name == name).get
}
fullOutput: Seq[AttributeReference],
@transient reader: DataSourceV2Reader) extends LeafExecNode with DataSourceReaderHolder {

override def references: AttributeSet = AttributeSet.empty

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,64 +29,8 @@ import org.apache.spark.sql.sources.v2.reader._
object DataSourceV2Strategy extends Strategy {
// TODO: write path
override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case PhysicalOperation(projects, filters, DataSourceV2Relation(output, reader)) =>
val stayUpFilters: Seq[Expression] = reader match {
case r: SupportsPushDownCatalystFilters =>
r.pushCatalystFilters(filters.toArray)

case r: SupportsPushDownFilters =>
// A map from original Catalyst expressions to corresponding translated data source
// filters. If a predicate is not in this map, it means it cannot be pushed down.
val translatedMap: Map[Expression, Filter] = filters.flatMap { p =>
DataSourceStrategy.translateFilter(p).map(f => p -> f)
}.toMap

// Catalyst predicate expressions that cannot be converted to data source filters.
val nonConvertiblePredicates = filters.filterNot(translatedMap.contains)

// Data source filters that cannot be pushed down. An unhandled filter means
// the data source cannot guarantee the rows returned can pass the filter.
// As a result we must return it so Spark can plan an extra filter operator.
val unhandledFilters = r.pushFilters(translatedMap.values.toArray).toSet
val unhandledPredicates = translatedMap.filter { case (_, f) =>
unhandledFilters.contains(f)
}.keys

nonConvertiblePredicates ++ unhandledPredicates

case _ => filters
}

val attrMap = AttributeMap(output.zip(output))
val projectSet = AttributeSet(projects.flatMap(_.references))
val filterSet = AttributeSet(stayUpFilters.flatMap(_.references))

// Match original case of attributes.
// TODO: nested fields pruning
val requiredColumns = (projectSet ++ filterSet).toSeq.map(attrMap)
reader match {
case r: SupportsPushDownRequiredColumns =>
r.pruneColumns(requiredColumns.toStructType)
case _ =>
}

val scan = DataSourceV2ScanExec(
output.toArray,
reader,
reader.readSchema(),
ExpressionSet(filters),
Nil)

val filterCondition = stayUpFilters.reduceLeftOption(And)
val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan)

val withProject = if (projects == withFilter.output) {
withFilter
} else {
ProjectExec(projects, withFilter)
}

withProject :: Nil
case DataSourceV2Relation(output, reader) =>
DataSourceV2ScanExec(output, reader) :: Nil

case _ => Nil
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/*
* 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.datasources.v2

import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeMap, Expression, NamedExpression, PredicateHelper}
import org.apache.spark.sql.catalyst.optimizer.RemoveRedundantProject
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.datasources.DataSourceStrategy
import org.apache.spark.sql.sources
import org.apache.spark.sql.sources.v2.reader._

/**
* Pushes down various operators to the underlying data source for better performance. Operators are
* being pushed down with a specific order. As an example, given a LIMIT has a FILTER child, you
* can't push down LIMIT if FILTER is not completely pushed down. When both are pushed down, the
* data source should execute FILTER before LIMIT. And required columns are calculated at the end,
* because when more operators are pushed down, we may need less columns at Spark side.
*/
object PushDownOperatorsToDataSource extends Rule[LogicalPlan] with PredicateHelper {
override def apply(plan: LogicalPlan): LogicalPlan = {
Copy link
Member

Choose a reason for hiding this comment

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

This is an optimizer rule? The input is a LogicalPlan and the output is still a LogicalPlan?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yea it's an optimizer rule run before planner

Copy link
Member

Choose a reason for hiding this comment

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

Can we add a test suite for the unit test cases of this rule?

// Note that, we need to collect the target operator along with PROJECT node, as PROJECT may
// appear in many places for column pruning.
// TODO: Ideally column pruning should be implemented via a plan property that is propagated
// top-down, then we can simplify the logic here and only collect target operators.
val filterPushed = plan transformUp {
case FilterAndProject(fields, condition, r @ DataSourceV2Relation(_, reader)) =>
// Non-deterministic expressions are stateful and we must keep the input sequence unchanged
// to avoid changing the result. This means, we can't evaluate the filter conditions that
// are after the first non-deterministic condition ahead. Here we only try to push down
// deterministic conditions that are before the first non-deterministic condition.
val (candidates, containingNonDeterministic) =
splitConjunctivePredicates(condition).span(_.deterministic)
Copy link
Contributor

Choose a reason for hiding this comment

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

It isn't immediately clear why you would use span here instead of partition. I think it is because span will produce all deterministic predicates that would be run before the first non-deterministic predicate in an in-order traversal of teh condition, right? If so, then a comment would be really useful to make this clear. I'd also like to see a comment about why deterministic predicates "after" the first non-deterministic predicate shouldn't be pushed down. An example would really help, too.


val stayUpFilters: Seq[Expression] = reader match {
case r: SupportsPushDownCatalystFilters =>
r.pushCatalystFilters(candidates.toArray)

case r: SupportsPushDownFilters =>
// A map from original Catalyst expressions to corresponding translated data source
// filters. If a predicate is not in this map, it means it cannot be pushed down.
val translatedMap: Map[Expression, sources.Filter] = candidates.flatMap { p =>
DataSourceStrategy.translateFilter(p).map(f => p -> f)
}.toMap

// Catalyst predicate expressions that cannot be converted to data source filters.
val nonConvertiblePredicates = candidates.filterNot(translatedMap.contains)

// Data source filters that cannot be pushed down. An unhandled filter means
// the data source cannot guarantee the rows returned can pass the filter.
// As a result we must return it so Spark can plan an extra filter operator.
val unhandledFilters = r.pushFilters(translatedMap.values.toArray).toSet
val unhandledPredicates = translatedMap.filter { case (_, f) =>
unhandledFilters.contains(f)
}.keys

nonConvertiblePredicates ++ unhandledPredicates

case _ => candidates
}

val filterCondition = (stayUpFilters ++ containingNonDeterministic).reduceLeftOption(And)
val withFilter = filterCondition.map(Filter(_, r)).getOrElse(r)
if (withFilter.output == fields) {
withFilter
} else {
Project(fields, withFilter)
}
}

// TODO: add more push down rules.

// TODO: nested fields pruning
def pushDownRequiredColumns(plan: LogicalPlan, requiredByParent: Seq[Attribute]): Unit = {
plan match {
case Project(projectList, child) =>
val required = projectList.filter(requiredByParent.contains).flatMap(_.references)
pushDownRequiredColumns(child, required)

case Filter(condition, child) =>
val required = requiredByParent ++ condition.references
pushDownRequiredColumns(child, required)

case DataSourceV2Relation(fullOutput, reader) => reader match {
case r: SupportsPushDownRequiredColumns =>
// Match original case of attributes.
val attrMap = AttributeMap(fullOutput.zip(fullOutput))
val requiredColumns = requiredByParent.map(attrMap)
r.pruneColumns(requiredColumns.toStructType)
case _ =>
}

// TODO: there may be more operators can be used to calculate required columns, we can add
// more and more in the future.
case _ => plan.children.foreach(child => pushDownRequiredColumns(child, child.output))
Copy link
Contributor

Choose a reason for hiding this comment

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

How do we know that there aren't more cases that need to be supported?

}
}

pushDownRequiredColumns(filterPushed, filterPushed.output)
// After column pruning, we may have redundant PROJECT nodes in the query plan, remove them.
RemoveRedundantProject(filterPushed)
}

/**
* Finds a Filter node(with an optional Project child) above data source relation.
*/
object FilterAndProject {
// returns the project list, the filter condition and the data source relation.
def unapply(plan: LogicalPlan)
: Option[(Seq[NamedExpression], Expression, DataSourceV2Relation)] = plan match {

case Filter(condition, r: DataSourceV2Relation) => Some((r.output, condition, r))

case Filter(condition, Project(fields, r: DataSourceV2Relation))
if fields.forall(_.deterministic) =>
val attributeMap = AttributeMap(fields.map(e => e.toAttribute -> e))
val substituted = condition.transform {
case a: Attribute => attributeMap.getOrElse(a, a)
}
Some((fields, substituted, r))

case _ => None
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,11 @@ public Filter[] pushFilters(Filter[] filters) {
return new Filter[0];
}

@Override
public Filter[] pushedFilters() {
return filters;
}

@Override
public List<ReadTask<Row>> createReadTasks() {
List<ReadTask<Row>> res = new ArrayList<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,8 @@ class AdvancedDataSourceV2 extends DataSourceV2 with ReadSupport {
Array.empty
}

override def pushedFilters(): Array[Filter] = filters

override def readSchema(): StructType = {
requiredSchema
}
Expand Down