Repository: spark
Updated Branches:
  refs/heads/master 9cc74f95e -> b1e5281c5


[SPARK-12639][SQL] Mark Filters Fully Handled By Sources with *

## What changes were proposed in this pull request?

In order to make it clear which filters are fully handled by the
underlying datasource we will mark them with an *. This will give a
clear visual queue to users that the filter is being treated differently
by catalyst than filters which are just presented to the underlying
DataSource.

Examples from the FilteredScanSuite, in this example `c IN (...)` is handled by 
the source, `b < ...` is not
### Before
```
//SELECT a FROM oneToTenFiltered WHERE a + b > 9 AND b < 16 AND c IN 
('bbbbbBBBBB', 'cccccCCCCC', 'dddddDDDDD', 'foo')
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), 
In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

### After
```
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), 
*In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

## How was the this patch tested?

Manually tested with the Spark Cassandra Connector, a source which fully 
handles underlying filters. Now fully handled filters appear with an * next to 
their names. I can add an automated test as well if requested

Post 1.6.1
Tested by modifying the FilteredScanSuite to run explains.

Author: Russell Spitzer <russell.spit...@gmail.com>

Closes #11317 from RussellSpitzer/SPARK-12639-Star.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b1e5281c
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b1e5281c
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b1e5281c

Branch: refs/heads/master
Commit: b1e5281c5cb429e338c3719c13c0b93078d7312a
Parents: 9cc74f9
Author: Russell Spitzer <russell.spit...@gmail.com>
Authored: Mon Jul 11 21:40:09 2016 -0700
Committer: Yin Huai <yh...@databricks.com>
Committed: Mon Jul 11 21:40:09 2016 -0700

----------------------------------------------------------------------
 .../datasources/DataSourceStrategy.scala        | 26 ++++++++++++++------
 .../parquet/ParquetFilterSuite.scala            |  2 +-
 .../spark/sql/hive/orc/OrcFilterSuite.scala     |  4 +--
 3 files changed, 21 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/b1e5281c/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index 15c0ac7..0841636 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -332,7 +332,8 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
       case a: AttributeReference => relation.attributeMap(a) // Match original 
case of attributes.
     }}
 
-    val (unhandledPredicates, pushedFilters) = 
selectFilters(relation.relation, candidatePredicates)
+    val (unhandledPredicates, pushedFilters, handledFilters) =
+      selectFilters(relation.relation, candidatePredicates)
 
     // A set of column attributes that are only referenced by pushed down 
filters.  We can eliminate
     // them from requested columns.
@@ -349,8 +350,13 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
 
     val metadata: Map[String, String] = {
       val pairs = ArrayBuffer.empty[(String, String)]
+
+      // Mark filters which are handled by the underlying DataSource with an 
Astrisk
       if (pushedFilters.nonEmpty) {
-        pairs += (PUSHED_FILTERS -> pushedFilters.mkString("[", ", ", "]"))
+        val markedFilters = for (filter <- pushedFilters) yield {
+            if (handledFilters.contains(filter)) s"*$filter" else s"$filter"
+        }
+        pairs += (PUSHED_FILTERS -> markedFilters.mkString("[", ", ", "]"))
       }
       pairs.toMap
     }
@@ -492,13 +498,16 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
    * Selects Catalyst predicate [[Expression]]s which are convertible into 
data source [[Filter]]s
    * and can be handled by `relation`.
    *
-   * @return A pair of `Seq[Expression]` and `Seq[Filter]`. The first element 
contains all Catalyst
-   *         predicate [[Expression]]s that are either not convertible or 
cannot be handled by
-   *         `relation`. The second element contains all converted data source 
[[Filter]]s that
-   *         will be pushed down to the data source.
+   * @return A triplet of `Seq[Expression]`, `Seq[Filter]`, and `Seq[Filter]` 
. The first element
+   *         contains all Catalyst predicate [[Expression]]s that are either 
not convertible or
+   *         cannot be handled by `relation`. The second element contains all 
converted data source
+   *         [[Filter]]s that will be pushed down to the data source. The 
third element contains
+   *         all [[Filter]]s that are completely filtered at the DataSource.
    */
   protected[sql] def selectFilters(
-      relation: BaseRelation, predicates: Seq[Expression]): (Seq[Expression], 
Seq[Filter]) = {
+    relation: BaseRelation,
+    predicates: Seq[Expression]): (Seq[Expression], Seq[Filter], Set[Filter]) 
= {
+
     // For conciseness, all Catalyst filter expressions of type 
`expressions.Expression` below are
     // called `predicate`s, while all data source filters of type 
`sources.Filter` are simply called
     // `filter`s.
@@ -521,7 +530,8 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
     val unhandledPredicates = translatedMap.filter { case (p, f) =>
       unhandledFilters.contains(f)
     }.keys
+    val handledFilters = pushedFilters.toSet -- unhandledFilters
 
-    (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters)
+    (nonconvertiblePredicates ++ unhandledPredicates, pushedFilters, 
handledFilters)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/b1e5281c/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
index f59d474..d846b27 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
@@ -69,7 +69,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest 
with SharedSQLContex
         }.flatten.reduceLeftOption(_ && _)
         assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from 
the given query")
 
-        val (_, selectedFilters) =
+        val (_, selectedFilters, _) =
           DataSourceStrategy.selectFilters(maybeRelation.get, 
maybeAnalyzedPredicate.toSeq)
         assert(selectedFilters.nonEmpty, "No filter is pushed down")
 

http://git-wip-us.apache.org/repos/asf/spark/blob/b1e5281c/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala
index 7a30e54..471192a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcFilterSuite.scala
@@ -51,7 +51,7 @@ class OrcFilterSuite extends QueryTest with OrcTest {
     }.flatten.reduceLeftOption(_ && _)
     assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the 
given query")
 
-    val (_, selectedFilters) =
+    val (_, selectedFilters, _) =
       DataSourceStrategy.selectFilters(maybeRelation.get, 
maybeAnalyzedPredicate.toSeq)
     assert(selectedFilters.nonEmpty, "No filter is pushed down")
 
@@ -95,7 +95,7 @@ class OrcFilterSuite extends QueryTest with OrcTest {
     }.flatten.reduceLeftOption(_ && _)
     assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the 
given query")
 
-    val (_, selectedFilters) =
+    val (_, selectedFilters, _) =
       DataSourceStrategy.selectFilters(maybeRelation.get, 
maybeAnalyzedPredicate.toSeq)
     assert(selectedFilters.nonEmpty, "No filter is pushed down")
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to