[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-12 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/19424


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-12 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r144274334
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceReaderHolder.scala
 ---
@@ -0,0 +1,68 @@
+/*
+ * 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 with customized 
equals/hashCode methods.
+ */
+trait DataSourceReaderHolder {
+
+  /**
+   * The full output of the data source reader, without column pruning.
+   */
+  def fullOutput: Seq[AttributeReference]
+
+  /**
+   * The held data source reader.
+   */
+  def reader: DataSourceV2Reader
+
+  /**
+   * The metadata of this data source reader that can be used for equality 
test.
+   */
+  private def metadata: Seq[Any] = {
+val filters: Any = reader match {
+  case s: SupportsPushDownCatalystFilters => 
s.pushedCatalystFilters().toSet
+  case s: SupportsPushDownFilters => s.pushedFilters().toSet
+  case _ => Nil
+}
+Seq(fullOutput, reader.getClass, reader.readSchema(), filters)
+  }
+
+  def canEqual(other: Any): Boolean
+
+  override def equals(other: Any): Boolean = other match {
+case other: DataSourceReaderHolder =>
+  canEqual(other) && metadata.length == other.metadata.length &&
+metadata.zip(other.metadata).forall { case (l, r) => l == r }
+case _ => false
+  }
+
+  override def hashCode(): Int = {
+metadata.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
--- End diff --

These names should already be normalized before reaching here.


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-12 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r144274528
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java
 ---
@@ -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;
--- End diff --

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


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-12 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r144234488
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java
 ---
@@ -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;
--- End diff --

Seems we don't use `Expression` here?


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-12 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r144233678
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceReaderHolder.scala
 ---
@@ -0,0 +1,68 @@
+/*
+ * 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 with customized 
equals/hashCode methods.
+ */
+trait DataSourceReaderHolder {
+
+  /**
+   * The full output of the data source reader, without column pruning.
+   */
+  def fullOutput: Seq[AttributeReference]
+
+  /**
+   * The held data source reader.
+   */
+  def reader: DataSourceV2Reader
+
+  /**
+   * The metadata of this data source reader that can be used for equality 
test.
+   */
+  private def metadata: Seq[Any] = {
+val filters: Any = reader match {
+  case s: SupportsPushDownCatalystFilters => 
s.pushedCatalystFilters().toSet
+  case s: SupportsPushDownFilters => s.pushedFilters().toSet
+  case _ => Nil
+}
+Seq(fullOutput, reader.getClass, reader.readSchema(), filters)
+  }
+
+  def canEqual(other: Any): Boolean
+
+  override def equals(other: Any): Boolean = other match {
+case other: DataSourceReaderHolder =>
+  canEqual(other) && metadata.length == other.metadata.length &&
+metadata.zip(other.metadata).forall { case (l, r) => l == r }
+case _ => false
+  }
+
+  override def hashCode(): Int = {
+metadata.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
--- End diff --

Shall we use resolver instead of string comparison?


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-12 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r144209319
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceReaderHolder.scala
 ---
@@ -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.
+ */
+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
--- End diff --

good catch!


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r144111358
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceReaderHolder.scala
 ---
@@ -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.
+ */
+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
--- End diff --

The evaluation order of these filters must be the same?


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-11 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r144109443
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceReaderHolder.scala
 ---
@@ -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.
--- End diff --

`defines` -> `with a customized`


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-09 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r143597547
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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, PredicateHelper}
+import org.apache.spark.sql.catalyst.optimizer.{PushDownPredicate, 
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 = {
+// make sure filters are at very bottom.
+val prepared = PushDownPredicate(plan)
+val afterPushDown = prepared transformUp {
+  case Filter(condition, r @ DataSourceV2Relation(_, reader)) =>
+val (candidates, containingNonDeterministic) =
+  splitConjunctivePredicates(condition).span(_.deterministic)
+
+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)
+filterCondition.map(Filter(_, r)).getOrElse(r)
+
+  // 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 = re

[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-09 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r143593605
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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, PredicateHelper}
+import org.apache.spark.sql.catalyst.optimizer.{PushDownPredicate, 
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 = {
+// make sure filters are at very bottom.
+val prepared = PushDownPredicate(plan)
+val afterPushDown = prepared transformUp {
+  case Filter(condition, r @ DataSourceV2Relation(_, reader)) =>
+val (candidates, containingNonDeterministic) =
+  splitConjunctivePredicates(condition).span(_.deterministic)
--- End diff --

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.


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-09 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r143597669
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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, PredicateHelper}
+import org.apache.spark.sql.catalyst.optimizer.{PushDownPredicate, 
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 = {
+// make sure filters are at very bottom.
+val prepared = PushDownPredicate(plan)
+val afterPushDown = prepared transformUp {
+  case Filter(condition, r @ DataSourceV2Relation(_, reader)) =>
+val (candidates, containingNonDeterministic) =
+  splitConjunctivePredicates(condition).span(_.deterministic)
+
+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)
+filterCondition.map(Filter(_, r)).getOrElse(r)
+
+  // 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 = re

[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-09 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r143591559
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,104 @@
+/*
+ * 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, PredicateHelper}
+import org.apache.spark.sql.catalyst.optimizer.{PushDownPredicate, 
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 = {
+// make sure filters are at very bottom.
+val prepared = PushDownPredicate(plan)
--- End diff --

Why apply this rule one more time? Is there reason to suspect that 
predicates won't already be pushed and that one more run would be worth it?


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r143072487
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,99 @@
+/*
+ * 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, AttributeMap, 
AttributeSet, Expression, ExpressionSet}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+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. We classify
+ * operators into different layers, operators in the same layer are 
orderless, i.e. the query result
+ * won't change if we switch the operators within a layer(e.g. we can 
switch the order of predicates
+ * and required columns). The operators in layer N can only be pushed down 
if operators in layer N-1
+ * that above the data source relation are all pushed down. As an example, 
you can't push down limit
+ * if a filter below limit is not pushed down.
+ *
+ * Current operator push down layers:
+ *   layer 1: predicates, required columns.
+ */
+object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
--- End diff --

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


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-04 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r142838899
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,99 @@
+/*
+ * 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, AttributeMap, 
AttributeSet, Expression, ExpressionSet}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+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. We classify
+ * operators into different layers, operators in the same layer are 
orderless, i.e. the query result
+ * won't change if we switch the operators within a layer(e.g. we can 
switch the order of predicates
+ * and required columns). The operators in layer N can only be pushed down 
if operators in layer N-1
+ * that above the data source relation are all pushed down. As an example, 
you can't push down limit
+ * if a filter below limit is not pushed down.
+ *
+ * Current operator push down layers:
+ *   layer 1: predicates, required columns.
+ */
+object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
--- End diff --

yea it's an optimizer rule run before planner


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-04 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r142800670
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,99 @@
+/*
+ * 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, AttributeMap, 
AttributeSet, Expression, ExpressionSet}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+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. We classify
+ * operators into different layers, operators in the same layer are 
orderless, i.e. the query result
+ * won't change if we switch the operators within a layer(e.g. we can 
switch the order of predicates
+ * and required columns). The operators in layer N can only be pushed down 
if operators in layer N-1
+ * that above the data source relation are all pushed down. As an example, 
you can't push down limit
+ * if a filter below limit is not pushed down.
--- End diff --

> 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.


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-04 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r142801593
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -32,13 +32,12 @@ import org.apache.spark.sql.types.StructType
 case class DataSourceV2ScanExec(
--- End diff --

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


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-04 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19424#discussion_r142806719
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownOperatorsToDataSource.scala
 ---
@@ -0,0 +1,99 @@
+/*
+ * 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, AttributeMap, 
AttributeSet, Expression, ExpressionSet}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+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. We classify
+ * operators into different layers, operators in the same layer are 
orderless, i.e. the query result
+ * won't change if we switch the operators within a layer(e.g. we can 
switch the order of predicates
+ * and required columns). The operators in layer N can only be pushed down 
if operators in layer N-1
+ * that above the data source relation are all pushed down. As an example, 
you can't push down limit
+ * if a filter below limit is not pushed down.
+ *
+ * Current operator push down layers:
+ *   layer 1: predicates, required columns.
+ */
+object PushDownOperatorsToDataSource extends Rule[LogicalPlan] {
+  override def apply(plan: LogicalPlan): LogicalPlan = {
--- End diff --

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


---

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



[GitHub] spark pull request #19424: [SPARK-22197][SQL] push down operators to data so...

2017-10-03 Thread cloud-fan
GitHub user cloud-fan opened a pull request:

https://github.com/apache/spark/pull/19424

[SPARK-22197][SQL] push down operators to data source before planning

## What changes were proposed in this pull request?

As we discussed in 
https://github.com/apache/spark/pull/19136#discussion_r137023744 , we should 
push down operators to data source before planning, so that data source can 
report statistics more accurate.

## How was this patch tested?

existing tests.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/cloud-fan/spark follow

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/19424.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #19424


commit 571fad95c2dac6f70840da9c68cfb1be1358ecaf
Author: Wenchen Fan 
Date:   2017-09-27T12:50:15Z

improve documents and minor clean up

commit 75457f608b9068ad1b3b3eb129f14d4e1b4ed946
Author: Wenchen Fan 
Date:   2017-10-04T01:37:25Z

push down operators to data source before planning




---

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