You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/02/28 00:02:00 UTC

[GitHub] [spark] dbtsai opened a new pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

dbtsai opened a new pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728
 
 
   ### What changes were proposed in this pull request?
   1. `DataSourceStrategy.scala` is extended to create `org.apache.spark.sql.sources.Filter` from nested expressions. 
   2. Translation from nested `org.apache.spark.sql.sources.Filter` to `org.apache.parquet.filter2.predicate.FilterPredicate` is implemented to support nested predicate pushdown for Parquet.
   
   ### Why are the changes needed?
   Better performance for handling nested predicate pushdown.
   
   ### Does this PR introduce any user-facing change?
   No
   
   
   ### How was this patch tested?
   New tests are added.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603633607
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25015/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-607021780
 
 
   For the worst case, we don't see extra overhead from parquet side. I agree that we should have benchmark suite for best (filtering out everything in a row group) and the worst case that nothing is filtered out in a row group.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592806995
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119107/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398317799
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -103,22 +107,42 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df)
   }
 
-  private def checkBinaryFilterPredicate
 
 Review comment:
   @cloud-fan 
   Since `checkFilterPredicate` works for binary data in the current implementation, `checkBinaryFilterPredicate` can be deleted.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387227741
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala
 ##########
 @@ -63,13 +63,38 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest {
       (f: String => Unit): Unit = withDataSourceFile(data)(f)
 
   /**
-   * Writes `data` to a Parquet file and reads it back as a [[DataFrame]],
+   * Writes `data` objects to a Parquet file and reads it back as a [[DataFrame]],
    * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
    */
-  protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag]
+  protected def withParquetDFfromObjs[T <: Product: ClassTag: TypeTag]
       (data: Seq[T], testVectorized: Boolean = true)
       (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f)
 
+  /**
+   * Writes `df` dataframe to a Parquet file and reads it back as a [[DataFrame]],
+   * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
+   */
+  protected def withParquetDFfromDF[T <: Product: ClassTag: TypeTag]
+      (df: DataFrame, testVectorized: Boolean = true)
+      (f: DataFrame => Unit): Unit = {
+    withTempPath { file =>
+      df.write.format(dataSourceName).save(file.getCanonicalPath)
+      readFile(file.getCanonicalPath, testVectorized)(f)
+    }
+  }
+
+  /**
+   * Writes `df` to a Parquet file and reads it back as a [[DataFrame]],
+   * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
+   */
+  protected def toParquetDataFrame(df: DataFrame, testVectorized: Boolean = true)
 
 Review comment:
   Is this used? Shall we remove this to minimize this PR?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r397600752
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -40,12 +45,32 @@ sealed abstract class Filter {
     case f: Filter => f.references
     case _ => Array.empty
   }
+
+  /**
+   * List of columns that are referenced by this filter.
+   *
+   * @return each element is a column name as an array of string multi-identifier
+   * @since 3.0.0
+   */
+  def V2references: Array[Array[String]] = {
 
 Review comment:
   done. thanks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385928057
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala
 ##########
 @@ -22,69 +22,119 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.PlanTest
 import org.apache.spark.sql.sources
 import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
 
 class DataSourceStrategySuite extends PlanTest with SharedSparkSession {
 
-  test("translate simple expression") {
-    val attrInt = 'cint.int
-    val attrStr = 'cstr.string
-
-    testTranslateFilter(EqualTo(attrInt, 1), Some(sources.EqualTo("cint", 1)))
-    testTranslateFilter(EqualTo(1, attrInt), Some(sources.EqualTo("cint", 1)))
+  val attrInts = Seq(
+    'cint.int,
+    Symbol("c.int").int,
+    GetStructField('a.struct(StructType(
+      StructField("cstr", StringType, nullable = true) ::
+        StructField("cint", IntegerType, nullable = true) :: Nil)), 1, None),
+    GetStructField('a.struct(StructType(
+      StructField("c.int", IntegerType, nullable = true) ::
+        StructField("cstr", StringType, nullable = true) :: Nil)), 0, None),
+    GetStructField(Symbol("a.b").struct(StructType(
+      StructField("cstr1", StringType, nullable = true) ::
+        StructField("cstr2", StringType, nullable = true) ::
+        StructField("cint", IntegerType, nullable = true) :: Nil)), 2, None),
+    GetStructField(Symbol("a.b").struct(StructType(
+      StructField("c.int", IntegerType, nullable = true) :: Nil)), 0, None)
+  ).zip(Seq(
+    "cint",
+    "`c.int`", // single level column that contains `dot` in name
+    "a.cint", // two level nested column
+    "a.`c.int`", // two level nested column, and nested level contains `dot`
+    "`a.b`.cint", // two level nested column, and top level contains `dot`
+    "`a.b`.`c.int`" // two level nested column, and both levels contain `dot`
 
 Review comment:
   Nit: I think we can add a test case with more than 2 level nested columns here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597349207
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119627/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594977808
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24088/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387299506
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ##########
 @@ -437,61 +437,74 @@ object DataSourceStrategy {
     }
   }
 
+  /**
+   * Find the column name of an expression that can be pushed down.
+   */
+  private[sql] def pushDownColName(e: Expression): Option[String] = {
+    import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+    def helper(e: Expression): Option[Seq[String]] = e match {
+      case a: Attribute => Some(Seq(a.name))
+      case s: GetStructField => helper(s.child).map(_ :+ s.childSchema(s.ordinal).name)
+      case _ => None
+    }
+    helper(e).map(_.quoted)
+  }
+
   private def translateLeafNodeFilter(predicate: Expression): Option[Filter] = predicate match {
-    case expressions.EqualTo(a: Attribute, Literal(v, t)) =>
-      Some(sources.EqualTo(a.name, convertToScala(v, t)))
-    case expressions.EqualTo(Literal(v, t), a: Attribute) =>
-      Some(sources.EqualTo(a.name, convertToScala(v, t)))
-
-    case expressions.EqualNullSafe(a: Attribute, Literal(v, t)) =>
-      Some(sources.EqualNullSafe(a.name, convertToScala(v, t)))
-    case expressions.EqualNullSafe(Literal(v, t), a: Attribute) =>
-      Some(sources.EqualNullSafe(a.name, convertToScala(v, t)))
-
-    case expressions.GreaterThan(a: Attribute, Literal(v, t)) =>
-      Some(sources.GreaterThan(a.name, convertToScala(v, t)))
-    case expressions.GreaterThan(Literal(v, t), a: Attribute) =>
-      Some(sources.LessThan(a.name, convertToScala(v, t)))
-
-    case expressions.LessThan(a: Attribute, Literal(v, t)) =>
-      Some(sources.LessThan(a.name, convertToScala(v, t)))
-    case expressions.LessThan(Literal(v, t), a: Attribute) =>
-      Some(sources.GreaterThan(a.name, convertToScala(v, t)))
-
-    case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, t)) =>
-      Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t)))
-    case expressions.GreaterThanOrEqual(Literal(v, t), a: Attribute) =>
-      Some(sources.LessThanOrEqual(a.name, convertToScala(v, t)))
-
-    case expressions.LessThanOrEqual(a: Attribute, Literal(v, t)) =>
-      Some(sources.LessThanOrEqual(a.name, convertToScala(v, t)))
-    case expressions.LessThanOrEqual(Literal(v, t), a: Attribute) =>
-      Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t)))
-
-    case expressions.InSet(a: Attribute, set) =>
-      val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType)
-      Some(sources.In(a.name, set.toArray.map(toScala)))
+    case expressions.EqualTo(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.EqualTo(_, convertToScala(v, t)))
+    case expressions.EqualTo(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.EqualTo(_, convertToScala(v, t)))
+
+    case expressions.EqualNullSafe(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.EqualNullSafe(_, convertToScala(v, t)))
+    case expressions.EqualNullSafe(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.EqualNullSafe(_, convertToScala(v, t)))
+
+    case expressions.GreaterThan(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.GreaterThan(_, convertToScala(v, t)))
+    case expressions.GreaterThan(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.LessThan(_, convertToScala(v, t)))
+
+    case expressions.LessThan(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.LessThan(_, convertToScala(v, t)))
+    case expressions.LessThan(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.GreaterThan(_, convertToScala(v, t)))
+
+    case expressions.GreaterThanOrEqual(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.GreaterThanOrEqual(_, convertToScala(v, t)))
+    case expressions.GreaterThanOrEqual(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.LessThanOrEqual(_, convertToScala(v, t)))
+
+    case expressions.LessThanOrEqual(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.LessThanOrEqual(_, convertToScala(v, t)))
+    case expressions.LessThanOrEqual(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.GreaterThanOrEqual(_, convertToScala(v, t)))
+
+    case expressions.InSet(e: Expression, set) =>
+      val toScala = CatalystTypeConverters.createToScalaConverter(e.dataType)
+      pushDownColName(e).map(sources.In(_, set.toArray.map(toScala)))
 
     // Because we only convert In to InSet in Optimizer when there are more than certain
     // items. So it is possible we still get an In expression here that needs to be pushed
     // down.
-    case expressions.In(a: Attribute, list) if list.forall(_.isInstanceOf[Literal]) =>
+    case expressions.In(e: Expression, list) if list.forall(_.isInstanceOf[Literal]) =>
       val hSet = list.map(_.eval(EmptyRow))
-      val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType)
-      Some(sources.In(a.name, hSet.toArray.map(toScala)))
+      val toScala = CatalystTypeConverters.createToScalaConverter(e.dataType)
 
 Review comment:
   ditto. Let's wrap the body of `case` statement with `pushDownColName(e).map`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594256425
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595045854
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119350/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604252178
 
 
   Looks fine to me too

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385923179
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -40,6 +40,14 @@ sealed abstract class Filter {
     case f: Filter => f.references
     case _ => Array.empty
   }
+
+  /**
+   * If any of the references of this filter contains nested field
+   * @since 3.2.0
 
 Review comment:
   +1, seems to be 3.1.0

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592829841
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592248472
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595056218
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592862854
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119109/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603631938
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25014/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592297239
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603006968
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120225/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603006961
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603676929
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594256140
 
 
   **[Test build #119256 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119256/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592241013
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119057/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592784508
 
 
   **[Test build #119107 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119107/testReport)** for PR 27728 at commit [`2da3737`](https://github.com/apache/spark/commit/2da3737b4f0560e3919f1098b108e7047ea287fb).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-601920214
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603631938
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25014/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385977355
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 ##########
 @@ -466,13 +485,8 @@ class ParquetFilters(
     case _ => false
   }
 
-  // Parquet does not allow dots in the column name because dots are used as a column path
-  // delimiter. Since Parquet 1.8.2 (PARQUET-389), Parquet accepts the filter predicates
-  // with missing columns. The incorrect results could be got from Parquet when we push down
-  // filters for the column having dots in the names. Thus, we do not push down such filters.
-  // See SPARK-20364.
   private def canMakeFilterOn(name: String, value: Any): Boolean = {
-    nameToParquetField.contains(name) && !name.contains(".") && valueCanMakeFilterOn(name, value)
+    nameToParquetField.contains(name) && valueCanMakeFilterOn(name, value)
 
 Review comment:
   Based on the previous comment, we still need to prevent ``` `a.b`.c.d ```?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r397657619
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala
 ##########
 @@ -62,13 +62,21 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest {
       (data: Seq[T])
       (f: String => Unit): Unit = withDataSourceFile(data)(f)
 
+  protected def toDF[T <: Product: ClassTag: TypeTag](data: Seq[T]): DataFrame = {
 
 Review comment:
   instead of adding this new util function, shall we add an overload of `withParquetDataFrame` that still takes `Seq`? Then we can avoid changing some test cases.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604410058
 
 
   **[Test build #120422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120422/testReport)** for PR 27728 at commit [`5fd97c0`](https://github.com/apache/spark/commit/5fd97c0a90eb1885a93fffb9d04a262b35f62bc3).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r386739846
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +134,8 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  def unquote(name: String): Seq[String] = {
+    new CatalystSqlParser(SQLConf.get).parseMultipartIdentifier(name)
 
 Review comment:
   How about 
   
   ```scala
     private lazy val catalystSqlParser = new CatalystSqlParser(SQLConf.get)
   
     def unquote(name: String): Seq[String] = {
       catalystSqlParser.parseMultipartIdentifier(name)
     }
   ```
   for now until we find a good place to have an internal instance of `catalystSqlParser`?
   
   The concern is that if we have it in the singleton, what if the `SQLConf` is changed?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594977802
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594215476
 
 
   **[Test build #119260 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119260/testReport)** for PR 27728 at commit [`3fde48c`](https://github.com/apache/spark/commit/3fde48c8e3cfc001b89f2f430b2bb0a84d0bdb91).
    * This patch **fails Scala style tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `public final class SparkFilterApi `

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604265583
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604328947
 
 
   **[Test build #120410 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120410/testReport)** for PR 27728 at commit [`5fd97c0`](https://github.com/apache/spark/commit/5fd97c0a90eb1885a93fffb9d04a262b35f62bc3).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r396839799
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala
 ##########
 @@ -45,4 +47,11 @@ trait OrcFiltersBase {
     case _: AtomicType => true
     case _ => false
   }
+
+  /**
+   * The key of the dataTypeMap will be quoted if it contains `dots`.
+   */
+  protected[orc] def quotedDataTypeMap(schema: StructType): Map[String, DataType] = {
 
 Review comment:
   qq: why is it protected?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r397655390
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -121,43 +121,81 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df)
   }
 
+  /**
+   * Takes single level `inputDF` dataframe to generate multi-level nested
+   * dataframes as new test data.
+   */
+  private def withNestedDataFrame(inputDF: DataFrame)
 
 Review comment:
   how about
   ```
   def withNestedDataFrame[T : TypeTag](data: Seq[Tuple1[T]])... {
     val df = data.toDF("temp")
   }
   ```
   
   Otherwise, the `inputDF` can be any schema and this method looks fragile.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592248105
 
 
   **[Test build #119060 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119060/testReport)** for PR 27728 at commit [`3a2fb39`](https://github.com/apache/spark/commit/3a2fb390975b97a60005d4df275bfddaf5d07a6d).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-601920214
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603006961
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603852404
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120323/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398738263
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -187,201 +209,273 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
   }
 
   test("filter pushdown - boolean") {
-    withParquetDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], Seq(Row(true), Row(false)))
-
-      checkFilterPredicate('_1 === true, classOf[Eq[_]], true)
-      checkFilterPredicate('_1 <=> true, classOf[Eq[_]], true)
-      checkFilterPredicate('_1 =!= true, classOf[NotEq[_]], false)
+    val data = (true :: false :: Nil).map(b => Tuple1.apply(Option(b)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val booleanAttr = df(colName).expr
+        assert(df(colName).expr.dataType === BooleanType)
+
+        checkFilterPredicate(booleanAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(booleanAttr.isNotNull, classOf[NotEq[_]],
+          Seq(Row(resultFun(true)), Row(resultFun(false))))
+
+        checkFilterPredicate(booleanAttr === true, classOf[Eq[_]], resultFun(true))
+        checkFilterPredicate(booleanAttr <=> true, classOf[Eq[_]], resultFun(true))
+        checkFilterPredicate(booleanAttr =!= true, classOf[NotEq[_]], resultFun(false))
+      }
     }
   }
 
   test("filter pushdown - tinyint") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toByte)))) { implicit df =>
-      assert(df.schema.head.dataType === ByteType)
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1.toByte, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1.toByte, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1.toByte, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2.toByte, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3.toByte, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1.toByte, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4.toByte, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1.toByte) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1.toByte) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2.toByte) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3.toByte) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1.toByte) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4.toByte) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4.toByte), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2.toByte || '_1 > 3.toByte,
-        classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toByte)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val tinyIntAttr = df(colName).expr
+        assert(df(colName).expr.dataType === ByteType)
+
+        checkFilterPredicate(tinyIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(tinyIntAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(tinyIntAttr === 1.toByte, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr <=> 1.toByte, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr =!= 1.toByte, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(tinyIntAttr < 2.toByte, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr > 3.toByte, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(tinyIntAttr <= 1.toByte, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr >= 4.toByte, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1.toByte) === tinyIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1.toByte) <=> tinyIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2.toByte) > tinyIntAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3.toByte) < tinyIntAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1.toByte) >= tinyIntAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4.toByte) <= tinyIntAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(tinyIntAttr < 4.toByte), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(tinyIntAttr < 2.toByte || tinyIntAttr > 3.toByte,
+          classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - smallint") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit df =>
-      assert(df.schema.head.dataType === ShortType)
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1.toShort, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1.toShort, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1.toShort, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2.toShort, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3.toShort, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1.toShort, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4.toShort, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1.toShort) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1.toShort) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2.toShort) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3.toShort) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1.toShort) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4.toShort) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4.toShort), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2.toShort || '_1 > 3.toShort,
-        classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toShort)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val smallIntAttr = df(colName).expr
+        assert(df(colName).expr.dataType === ShortType)
+
+        checkFilterPredicate(smallIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(smallIntAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(smallIntAttr === 1.toShort, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr <=> 1.toShort, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr =!= 1.toShort, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(smallIntAttr < 2.toShort, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr > 3.toShort, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(smallIntAttr <= 1.toShort, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr >= 4.toShort, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1.toShort) === smallIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1.toShort) <=> smallIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2.toShort) > smallIntAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3.toShort) < smallIntAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1.toShort) >= smallIntAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4.toShort) <= smallIntAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(smallIntAttr < 4.toShort), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(smallIntAttr < 2.toShort || smallIntAttr > 3.toShort,
+          classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - integer") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val intAttr = df(colName).expr
+        assert(df(colName).expr.dataType === IntegerType)
+
+        checkFilterPredicate(intAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(intAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(intAttr === 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(intAttr <=> 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(intAttr =!= 1, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(intAttr < 2, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(intAttr > 3, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(intAttr <= 1, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(intAttr >= 4, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1) === intAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1) <=> intAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2) > intAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3) < intAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1) >= intAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4) <= intAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(intAttr < 4), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(intAttr < 2 || intAttr > 3, classOf[Operators.Or],
+          Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - long") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toLong)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val longAttr = df(colName).expr
+        assert(df(colName).expr.dataType === LongType)
+
+        checkFilterPredicate(longAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(longAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(longAttr === 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(longAttr <=> 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(longAttr =!= 1, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(longAttr < 2, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(longAttr > 3, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(longAttr <= 1, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(longAttr >= 4, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1) === longAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1) <=> longAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2) > longAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3) < longAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1) >= longAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4) <= longAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(longAttr < 4), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(longAttr < 2 || longAttr > 3, classOf[Operators.Or],
+          Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - float") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toFloat)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val floatAttr = df(colName).expr
+        assert(df(colName).expr.dataType === FloatType)
+
+        checkFilterPredicate(floatAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(floatAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(floatAttr === 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(floatAttr <=> 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(floatAttr =!= 1, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(floatAttr < 2, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(floatAttr > 3, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(floatAttr <= 1, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(floatAttr >= 4, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1) === floatAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1) <=> floatAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2) > floatAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3) < floatAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1) >= floatAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4) <= floatAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(floatAttr < 4), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(floatAttr < 2 || floatAttr > 3, classOf[Operators.Or],
+          Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - double") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toDouble)))
+    import testImplicits._
 
 Review comment:
   Unfortunately, this implicit import will fail 
   ```scala
       } { implicit df =>
         checkFilterPredicate(
           '_1.startsWith("blah").asInstanceOf[Predicate],
           classOf[UserDefinedByInstance[_, _]],
   ```
   More specifically, the implicit conversion from `Symbols` to `Predicate`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593707022
 
 
   **[Test build #119192 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119192/testReport)** for PR 27728 at commit [`2eafaf3`](https://github.com/apache/spark/commit/2eafaf358f375a5508e123cb00889f0eda599a62).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597220562
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24356/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592243369
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597349207
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119627/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597224585
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24357/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592865772
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604265998
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593780084
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594164391
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119248/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595980652
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-596038593
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119492/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594113467
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23989/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592829841
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604408156
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593718921
 
 
   **[Test build #119194 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119194/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r386671859
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -40,6 +41,13 @@ sealed abstract class Filter {
     case f: Filter => f.references
     case _ => Array.empty
   }
+
+  /**
+   * If any of the references of this filter contains nested column
+   */
+  private[sql] def containNestedColumn: Boolean = {
 
 Review comment:
   Minor: other places (i.e. `Collection#contains`) normally use `contains` instead of `contain`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604578973
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594256324
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119256/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592793820
 
 
   **[Test build #119108 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119108/testReport)** for PR 27728 at commit [`784d372`](https://github.com/apache/spark/commit/784d372080e0417e3af8f5d246bccb179ff35bff).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592293046
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604286067
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25118/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-607011110
 
 
   It really depends on the data and if we can skip most of the row group by having a predicate in nested field. For our production jobs, we are able to get 20x speed-up in clock time, and 8x less data being read. I will create a synthetic benchmark for this feature.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r395819986
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -32,6 +33,7 @@ import org.apache.spark.annotation.{Evolving, Stable}
 sealed abstract class Filter {
   /**
    * List of columns that are referenced by this filter.
+   * Note that, if a column contains `dots` in name, it will be quoted to avoid confusion.
 
 Review comment:
   I agree that some downstream datasoruce implementation might not support column name containing `dots` or nested column predicate pushdown. However, we are talking about the contract of Spark's Filter API. The implementation of the translation logics for each datasources should take care of this properly instead, and that's why we have this abstraction. WDYT?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604249875
 
 
   **[Test build #120401 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120401/testReport)** for PR 27728 at commit [`f732664`](https://github.com/apache/spark/commit/f732664630c6465ef8d4ec58dbc3965fcee82f3b).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593711152
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23933/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398302492
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala
 ##########
 @@ -62,13 +62,21 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest {
       (data: Seq[T])
       (f: String => Unit): Unit = withDataSourceFile(data)(f)
 
+  protected def toDF[T <: Product: ClassTag: TypeTag](data: Seq[T]): DataFrame = {
 
 Review comment:
   The error is
   ```scala
   Error:(45, 20) in trait ParquetTest, multiple overloaded alternatives of method withParquetDataFrame define default arguments.
   private[sql] trait ParquetTest extends FileBasedDataSourceTest {
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398738263
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -187,201 +209,273 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
   }
 
   test("filter pushdown - boolean") {
-    withParquetDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], Seq(Row(true), Row(false)))
-
-      checkFilterPredicate('_1 === true, classOf[Eq[_]], true)
-      checkFilterPredicate('_1 <=> true, classOf[Eq[_]], true)
-      checkFilterPredicate('_1 =!= true, classOf[NotEq[_]], false)
+    val data = (true :: false :: Nil).map(b => Tuple1.apply(Option(b)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val booleanAttr = df(colName).expr
+        assert(df(colName).expr.dataType === BooleanType)
+
+        checkFilterPredicate(booleanAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(booleanAttr.isNotNull, classOf[NotEq[_]],
+          Seq(Row(resultFun(true)), Row(resultFun(false))))
+
+        checkFilterPredicate(booleanAttr === true, classOf[Eq[_]], resultFun(true))
+        checkFilterPredicate(booleanAttr <=> true, classOf[Eq[_]], resultFun(true))
+        checkFilterPredicate(booleanAttr =!= true, classOf[NotEq[_]], resultFun(false))
+      }
     }
   }
 
   test("filter pushdown - tinyint") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toByte)))) { implicit df =>
-      assert(df.schema.head.dataType === ByteType)
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1.toByte, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1.toByte, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1.toByte, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2.toByte, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3.toByte, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1.toByte, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4.toByte, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1.toByte) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1.toByte) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2.toByte) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3.toByte) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1.toByte) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4.toByte) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4.toByte), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2.toByte || '_1 > 3.toByte,
-        classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toByte)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val tinyIntAttr = df(colName).expr
+        assert(df(colName).expr.dataType === ByteType)
+
+        checkFilterPredicate(tinyIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(tinyIntAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(tinyIntAttr === 1.toByte, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr <=> 1.toByte, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr =!= 1.toByte, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(tinyIntAttr < 2.toByte, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr > 3.toByte, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(tinyIntAttr <= 1.toByte, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(tinyIntAttr >= 4.toByte, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1.toByte) === tinyIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1.toByte) <=> tinyIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2.toByte) > tinyIntAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3.toByte) < tinyIntAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1.toByte) >= tinyIntAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4.toByte) <= tinyIntAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(tinyIntAttr < 4.toByte), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(tinyIntAttr < 2.toByte || tinyIntAttr > 3.toByte,
+          classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - smallint") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toShort)))) { implicit df =>
-      assert(df.schema.head.dataType === ShortType)
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1.toShort, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1.toShort, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1.toShort, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2.toShort, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3.toShort, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1.toShort, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4.toShort, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1.toShort) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1.toShort) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2.toShort) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3.toShort) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1.toShort) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4.toShort) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4.toShort), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2.toShort || '_1 > 3.toShort,
-        classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toShort)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val smallIntAttr = df(colName).expr
+        assert(df(colName).expr.dataType === ShortType)
+
+        checkFilterPredicate(smallIntAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(smallIntAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(smallIntAttr === 1.toShort, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr <=> 1.toShort, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr =!= 1.toShort, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(smallIntAttr < 2.toShort, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr > 3.toShort, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(smallIntAttr <= 1.toShort, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(smallIntAttr >= 4.toShort, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1.toShort) === smallIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1.toShort) <=> smallIntAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2.toShort) > smallIntAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3.toShort) < smallIntAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1.toShort) >= smallIntAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4.toShort) <= smallIntAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(smallIntAttr < 4.toShort), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(smallIntAttr < 2.toShort || smallIntAttr > 3.toShort,
+          classOf[Operators.Or], Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - integer") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val intAttr = df(colName).expr
+        assert(df(colName).expr.dataType === IntegerType)
+
+        checkFilterPredicate(intAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(intAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(intAttr === 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(intAttr <=> 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(intAttr =!= 1, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(intAttr < 2, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(intAttr > 3, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(intAttr <= 1, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(intAttr >= 4, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1) === intAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1) <=> intAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2) > intAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3) < intAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1) >= intAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4) <= intAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(intAttr < 4), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(intAttr < 2 || intAttr > 3, classOf[Operators.Or],
+          Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - long") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toLong)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val longAttr = df(colName).expr
+        assert(df(colName).expr.dataType === LongType)
+
+        checkFilterPredicate(longAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(longAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(longAttr === 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(longAttr <=> 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(longAttr =!= 1, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(longAttr < 2, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(longAttr > 3, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(longAttr <= 1, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(longAttr >= 4, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1) === longAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1) <=> longAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2) > longAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3) < longAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1) >= longAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4) <= longAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(longAttr < 4), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(longAttr < 2 || longAttr > 3, classOf[Operators.Or],
+          Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - float") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toFloat)))
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (inputDF, colName, resultFun) =>
+      withParquetDataFrame(inputDF) { implicit df =>
+        val floatAttr = df(colName).expr
+        assert(df(colName).expr.dataType === FloatType)
+
+        checkFilterPredicate(floatAttr.isNull, classOf[Eq[_]], Seq.empty[Row])
+        checkFilterPredicate(floatAttr.isNotNull, classOf[NotEq[_]],
+          (1 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(floatAttr === 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(floatAttr <=> 1, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(floatAttr =!= 1, classOf[NotEq[_]],
+          (2 to 4).map(i => Row.apply(resultFun(i))))
+
+        checkFilterPredicate(floatAttr < 2, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(floatAttr > 3, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(floatAttr <= 1, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(floatAttr >= 4, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(Literal(1) === floatAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(1) <=> floatAttr, classOf[Eq[_]], resultFun(1))
+        checkFilterPredicate(Literal(2) > floatAttr, classOf[Lt[_]], resultFun(1))
+        checkFilterPredicate(Literal(3) < floatAttr, classOf[Gt[_]], resultFun(4))
+        checkFilterPredicate(Literal(1) >= floatAttr, classOf[LtEq[_]], resultFun(1))
+        checkFilterPredicate(Literal(4) <= floatAttr, classOf[GtEq[_]], resultFun(4))
+
+        checkFilterPredicate(!(floatAttr < 4), classOf[GtEq[_]], resultFun(4))
+        checkFilterPredicate(floatAttr < 2 || floatAttr > 3, classOf[Operators.Or],
+          Seq(Row(resultFun(1)), Row(resultFun(4))))
+      }
     }
   }
 
   test("filter pushdown - double") {
-    withParquetDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], (1 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 === 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 <=> 1, classOf[Eq[_]], 1)
-      checkFilterPredicate('_1 =!= 1, classOf[NotEq[_]], (2 to 4).map(Row.apply(_)))
-
-      checkFilterPredicate('_1 < 2, classOf[Lt[_]], 1)
-      checkFilterPredicate('_1 > 3, classOf[Gt[_]], 4)
-      checkFilterPredicate('_1 <= 1, classOf[LtEq[_]], 1)
-      checkFilterPredicate('_1 >= 4, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(Literal(1) === '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(1) <=> '_1, classOf[Eq[_]], 1)
-      checkFilterPredicate(Literal(2) > '_1, classOf[Lt[_]], 1)
-      checkFilterPredicate(Literal(3) < '_1, classOf[Gt[_]], 4)
-      checkFilterPredicate(Literal(1) >= '_1, classOf[LtEq[_]], 1)
-      checkFilterPredicate(Literal(4) <= '_1, classOf[GtEq[_]], 4)
-
-      checkFilterPredicate(!('_1 < 4), classOf[GtEq[_]], 4)
-      checkFilterPredicate('_1 < 2 || '_1 > 3, classOf[Operators.Or], Seq(Row(1), Row(4)))
+    val data = (1 to 4).map(i => Tuple1(Option(i.toDouble)))
+    import testImplicits._
 
 Review comment:
   Unfortunately, this implicit import will fail 
   ```scala
       } { implicit df =>
         checkFilterPredicate(
           '_1.startsWith("blah").asInstanceOf[Predicate],
           classOf[UserDefinedByInstance[_, _]],
   ```
   More specifically, the implicit conversion from `Symbols` to `Expression`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603071682
 
 
   Jenkins, test this again.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387245545
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala
 ##########
 @@ -63,13 +63,38 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest {
       (f: String => Unit): Unit = withDataSourceFile(data)(f)
 
   /**
-   * Writes `data` to a Parquet file and reads it back as a [[DataFrame]],
+   * Writes `data` objects to a Parquet file and reads it back as a [[DataFrame]],
    * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
    */
-  protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag]
+  protected def withParquetDFfromObjs[T <: Product: ClassTag: TypeTag]
       (data: Seq[T], testVectorized: Boolean = true)
       (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f)
 
+  /**
+   * Writes `df` dataframe to a Parquet file and reads it back as a [[DataFrame]],
+   * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
+   */
+  protected def withParquetDFfromDF[T <: Product: ClassTag: TypeTag]
+      (df: DataFrame, testVectorized: Boolean = true)
+      (f: DataFrame => Unit): Unit = {
+    withTempPath { file =>
+      df.write.format(dataSourceName).save(file.getCanonicalPath)
+      readFile(file.getCanonicalPath, testVectorized)(f)
+    }
+  }
+
+  /**
+   * Writes `df` to a Parquet file and reads it back as a [[DataFrame]],
+   * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
+   */
+  protected def toParquetDataFrame(df: DataFrame, testVectorized: Boolean = true)
 
 Review comment:
   They are used in couple places. I can submit another PR for this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594256425
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594251307
 
 
   **[Test build #119266 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119266/testReport)** for PR 27728 at commit [`2c65b17`](https://github.com/apache/spark/commit/2c65b1753de2306742c491ad34b610dd01294023).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593591634
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan closed pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398280601
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala
 ##########
 @@ -62,13 +62,21 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest {
       (data: Seq[T])
       (f: String => Unit): Unit = withDataSourceFile(data)(f)
 
+  protected def toDF[T <: Product: ClassTag: TypeTag](data: Seq[T]): DataFrame = {
 
 Review comment:
   I was thinking to do so, but surprisingly, overloading 
   ```scala
   protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag]
         (data: Seq[T], testVectorized: Boolean = true)
         (f: DataFrame => Unit): Unit
   ```
   and 
   ```scala
   protected def withParquetDataFrame(df: DataFrame, testVectorized: Boolean = true)
      (f: DataFrame => Unit): Unit
   ```
   is not allowed in Scala.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398310277
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -121,43 +121,81 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df)
   }
 
+  /**
+   * Takes single level `inputDF` dataframe to generate multi-level nested
+   * dataframes as new test data.
+   */
+  private def withNestedDataFrame(inputDF: DataFrame)
 
 Review comment:
   Okay, this is not easy since one of the test case is like
   ```scala
           val dataFrame = spark.createDataFrame(rdd, StructType.fromDDL(s"a decimal($precision, 2)"))
           withNestedDataFrame(dataFrame) { case (inputDF, pushDownColName, resultFun) =>
             withParquetDataFrame(inputDF) { implicit df =>
               val decimalAttr: Expression = df(pushDownColName).expr
               assert(df(pushDownColName).expr.dataType === DecimalType(precision, 2))
   ```
   , so the dataframe can not be constructed directly from `withNestedDataFrame[T](data: Seq[T])`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592240598
 
 
   **[Test build #119057 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119057/testReport)** for PR 27728 at commit [`79c2b29`](https://github.com/apache/spark/commit/79c2b29acf792c772866753913660fe69a42c279).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r396987546
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala
 ##########
 @@ -62,13 +62,21 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest {
       (data: Seq[T])
       (f: String => Unit): Unit = withDataSourceFile(data)(f)
 
+  protected def toDF[T <: Product: ClassTag: TypeTag](data: Seq[T]): DataFrame = {
+    spark.createDataFrame(data)
+  }
+
   /**
-   * Writes `data` to a Parquet file and reads it back as a [[DataFrame]],
+   * Writes `df` dataframe to a Parquet file and reads it back as a [[DataFrame]],
    * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
    */
-  protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag]
-      (data: Seq[T], testVectorized: Boolean = true)
-      (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f)
+  protected def withParquetDataFrame(df: DataFrame, testVectorized: Boolean = true)
 
 Review comment:
   Why do we need to change the test framework? We can support nested data structure with `Seq[T]`, like case class

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r386619204
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +132,9 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  def unqote(input: String): Seq[String] = {
 
 Review comment:
   Fixed. Thanks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592779540
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597349202
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603677055
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120307/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603689016
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25034/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r399127200
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
 ##########
 @@ -2049,6 +2049,17 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val NESTED_PREDICATE_PUSHDOWN_ENABLED =
+    buildConf("spark.sql.optimizer.nestedPredicatePushdown.enabled")
+      .internal()
+      .doc("When true, Spark tries to push down predicates for nested columns and or names " +
+        "containing `dots` to data sources. Currently, Parquet implements both optimizations " +
+        "while ORC only supports predicates for names containing `dots`. The other data sources" +
+        "don't support this feature yet.")
+      .version("3.0.0")
+      .booleanConf
+      .createWithDefault(true)
 
 Review comment:
   Sure, I think it should be fine.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604224801
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25103/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593783696
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592243375
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23804/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603688581
 
 
   **[Test build #120323 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120323/testReport)** for PR 27728 at commit [`44b310e`](https://github.com/apache/spark/commit/44b310e423250c4635943c8e66323f37f03ef75b).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603676929
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597348520
 
 
   **[Test build #119627 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119627/testReport)** for PR 27728 at commit [`393d88d`](https://github.com/apache/spark/commit/393d88d77786210996a288cf0b6df8a75d39ad4c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595980339
 
 
   **[Test build #119492 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119492/testReport)** for PR 27728 at commit [`078b5b5`](https://github.com/apache/spark/commit/078b5b517f3dc19b49a9d1103160d4e0093781b6).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r397754720
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
 ##########
 @@ -2049,6 +2049,17 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val NESTED_PREDICATE_PUSHDOWN_ENABLED =
+    buildConf("spark.sql.optimizer.nestedPredicatePushdown.enabled")
+      .internal()
+      .doc("When true, Spark tries to push down predicates for nested columns and or names " +
+        "containing `dots` to data sources. Currently, Parquet implements both optimizations " +
+        "while ORC only supports predicates for names containing `dots`. The other data sources" +
+        "don't support this feature yet.")
+      .version("3.0.0")
+      .booleanConf
+      .createWithDefault(true)
 
 Review comment:
   Making a configuration that takes a list of sources could be done separately in a separate PR to make this PR smaller.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604410635
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/25130/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594979832
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24090/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603677046
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603676405
 
 
   **[Test build #120304 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120304/testReport)** for PR 27728 at commit [`c40f5d0`](https://github.com/apache/spark/commit/c40f5d035b6a4f82f0238285203cdbf74217b100).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593779606
 
 
   **[Test build #119193 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119193/testReport)** for PR 27728 at commit [`c170288`](https://github.com/apache/spark/commit/c170288ea13f0b52042e0356eafc5dafb78a5040).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-602930355
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603852393
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604265589
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120401/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593681451
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593707022
 
 
   **[Test build #119192 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119192/testReport)** for PR 27728 at commit [`2eafaf3`](https://github.com/apache/spark/commit/2eafaf358f375a5508e123cb00889f0eda599a62).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603631586
 
 
   **[Test build #120304 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120304/testReport)** for PR 27728 at commit [`c40f5d0`](https://github.com/apache/spark/commit/c40f5d035b6a4f82f0238285203cdbf74217b100).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594215256
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23999/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592240946
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23802/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592784886
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593718921
 
 
   **[Test build #119194 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119194/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r396944064
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -32,6 +33,7 @@ import org.apache.spark.annotation.{Evolving, Stable}
 sealed abstract class Filter {
   /**
    * List of columns that are referenced by this filter.
+   * Note that, if a column contains `dots` in name, it will be quoted to avoid confusion.
 
 Review comment:
   As @cloud-fan suggested, I add a conf to disable the new behavior so people can still use their old DSv1 implementation while upgrading to the new behavior. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595045849
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-596038593
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119492/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gatorsmile commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r410830628
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ##########
 @@ -652,10 +652,19 @@ object DataSourceStrategy {
  */
 object PushableColumn {
   def unapply(e: Expression): Option[String] = {
-    def helper(e: Expression) = e match {
-      case a: Attribute => Some(a.name)
+    val nestedPredicatePushdownEnabled = SQLConf.get.nestedPredicatePushdownEnabled
+    import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+    def helper(e: Expression): Option[Seq[String]] = e match {
+      case a: Attribute =>
+        if (nestedPredicatePushdownEnabled || !a.name.contains(".")) {
+          Some(Seq(a.name))
+        } else {
+          None
+        }
+      case s: GetStructField if nestedPredicatePushdownEnabled =>
+        helper(s.child).map(_ :+ s.childSchema(s.ordinal).name)
       case _ => None
     }
-    helper(e)
+    helper(e).map(_.quoted)
 
 Review comment:
   I am afraid it might break the released external third-party connectors that might not be able to handle the quoted column names. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398988592
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
 ##########
 @@ -2049,6 +2049,17 @@ object SQLConf {
       .booleanConf
       .createWithDefault(true)
 
+  val NESTED_PREDICATE_PUSHDOWN_ENABLED =
+    buildConf("spark.sql.optimizer.nestedPredicatePushdown.enabled")
+      .internal()
+      .doc("When true, Spark tries to push down predicates for nested columns and or names " +
+        "containing `dots` to data sources. Currently, Parquet implements both optimizations " +
+        "while ORC only supports predicates for names containing `dots`. The other data sources" +
+        "don't support this feature yet.")
+      .version("3.0.0")
+      .booleanConf
+      .createWithDefault(true)
 
 Review comment:
   FYI @HeartSaVioR, are you interested in this followup out of curiosity?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592806991
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592797687
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387213723
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -128,37 +134,47 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     val ts3 = data(2)
     val ts4 = data(3)
 
-    withParquetDataFrame(data.map(i => Tuple1(i))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], data.map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 === ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 <=> ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 =!= ts1, classOf[NotEq[_]],
-        Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 < ts2, classOf[Lt[_]], ts1)
-      checkFilterPredicate('_1 > ts1, classOf[Gt[_]], Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-      checkFilterPredicate('_1 <= ts1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate('_1 >= ts4, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(Literal(ts1) === '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts1) <=> '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts2) > '_1, classOf[Lt[_]], ts1)
-      checkFilterPredicate(Literal(ts3) < '_1, classOf[Gt[_]], ts4)
-      checkFilterPredicate(Literal(ts1) >= '_1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate(Literal(ts4) <= '_1, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(!('_1 < ts4), classOf[GtEq[_]], ts4)
-      checkFilterPredicate('_1 < ts2 || '_1 > ts3, classOf[Operators.Or], Seq(Row(ts1), Row(ts4)))
-    }
-  }
-
-  private def testDecimalPushDown(data: DataFrame)(f: DataFrame => Unit): Unit = {
-    withTempPath { file =>
-      data.write.parquet(file.getCanonicalPath)
-      readParquetFile(file.toString)(f)
-    }
+    Seq(
+      (
+        spark.sqlContext.createDataFrame(data.map(x => ColA(Some(x)))),
+        "a", // zero nesting
+        (x: Any) => x),
+      (
+        spark.sqlContext.createDataFrame(data.map(x => ColA(Some(ColB(Some(x)))))),
+        "a.b", // one level nesting
+        (x: Any) => Row(x)),
+      (
+        spark.sqlContext.createDataFrame(data.map(x => ColA(Some(ColB(Some(ColC(Some(x)))))))),
 
 Review comment:
   ditto.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385927422
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -40,6 +40,14 @@ sealed abstract class Filter {
     case f: Filter => f.references
     case _ => Array.empty
   }
+
+  /**
+   * If any of the references of this filter contains nested field
+   * @since 3.2.0
+   */
+  def containNestedField: Boolean = {
+    this.references.exists(_.contains("."))
 
 Review comment:
   What if there is a column name containing `.`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592779547
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23848/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594977808
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24088/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592792264
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23851/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604577894
 
 
   **[Test build #120422 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120422/testReport)** for PR 27728 at commit [`5fd97c0`](https://github.com/apache/spark/commit/5fd97c0a90eb1885a93fffb9d04a262b35f62bc3).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385975753
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 ##########
 @@ -49,15 +49,29 @@ class ParquetFilters(
     pushDownInFilterThreshold: Int,
     caseSensitive: Boolean) {
   // A map which contains parquet field name and data type, if predicate push down applies.
-  private val nameToParquetField : Map[String, ParquetField] = {
-    // Here we don't flatten the fields in the nested schema but just look up through
-    // root fields. Currently, accessing to nested fields does not push down filters
-    // and it does not support to create filters for them.
-    val primitiveFields =
-    schema.getFields.asScala.filter(_.isPrimitive).map(_.asPrimitiveType()).map { f =>
-      f.getName -> ParquetField(f.getName,
-        ParquetSchemaType(f.getOriginalType,
-          f.getPrimitiveTypeName, f.getTypeLength, f.getDecimalMetadata))
+  private val nameToParquetField : Map[String, ParquetPrimitiveField] = {
 
 Review comment:
   Can we add short comment here explaining what this method returns, especially for nested case?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594256436
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119253/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604265589
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120401/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595056218
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592245780
 
 
   **[Test build #119059 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119059/testReport)** for PR 27728 at commit [`400aebe`](https://github.com/apache/spark/commit/400aebe9d2526609c0a107a93d849288cac20708).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592779540
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594112894
 
 
   **[Test build #119248 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119248/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592302034
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119060/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592297243
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119059/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592697068
 
 
   **[Test build #119102 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119102/testReport)** for PR 27728 at commit [`0ae261d`](https://github.com/apache/spark/commit/0ae261d58cc1e967e499f4b9481a143c6234e17b).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594251578
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604265500
 
 
   **[Test build #120401 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120401/testReport)** for PR 27728 at commit [`f732664`](https://github.com/apache/spark/commit/f732664630c6465ef8d4ec58dbc3965fcee82f3b).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604247843
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gatorsmile commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-607018589
 
 
   Do you know the perf number for the worst case (e.g., no row can be filtered out)? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594112894
 
 
   **[Test build #119248 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119248/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603633598
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594215503
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119260/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603676729
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r390850480
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +134,10 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  private lazy val catalystSqlParser = new CatalystSqlParser(SQLConf.get)
+
+  def unquote(name: String): Seq[String] = {
 
 Review comment:
   it's not really unquote. How about `parseColumnPath`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604247843
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594179434
 
 
   **[Test build #119256 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119256/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593591646
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23925/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593776687
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604265583
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592811208
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119104/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604329206
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120410/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604266007
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120393/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai edited a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai edited a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592241475
 
 
   cc @dongjoon-hyun @holdenk @rdblue @gatorsmile @cloud-fan @hvanhovell @viirya @MaxGekk @HyukjinKwon @aokolnychyi @JoshRosen @gengliangwang

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592254545
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r396840153
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala
 ##########
 @@ -45,4 +47,11 @@ trait OrcFiltersBase {
     case _: AtomicType => true
     case _ => false
   }
+
+  /**
+   * The key of the dataTypeMap will be quoted if it contains `dots`.
 
 Review comment:
   Seems we should rephrase `dataTypeMap` in this doc.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592829867
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119106/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r386668161
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +134,8 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  def unquote(name: String): Seq[String] = {
+    new CatalystSqlParser(SQLConf.get).parseMultipartIdentifier(name)
 
 Review comment:
   This shouldn't create a new parser each time it is used. Can you create one in a companion object?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603689007
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r386592603
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +132,9 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  def unqote(input: String): Seq[String] = {
 
 Review comment:
   Typo: should be `unquote`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594979832
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24090/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604578989
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/120422/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603633352
 
 
   **[Test build #120305 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120305/testReport)** for PR 27728 at commit [`e9944e5`](https://github.com/apache/spark/commit/e9944e5dbf96ef17a24b2109033c2884cde4d2ca).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385442599
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -1107,7 +1130,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     }
   }
 
-  test("SPARK-20364: Disable Parquet predicate pushdown for fields having dots in the names") {
+  test("SPARK-17636: Parquet predicate pushdown for fields having dots in the names") {
 
 Review comment:
   @HyukjinKwon this PR also fixes the old problem that we don't allow to pushdown a column name containing "dot".

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595980658
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24221/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592697598
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595980652
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592240598
 
 
   **[Test build #119057 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119057/testReport)** for PR 27728 at commit [`79c2b29`](https://github.com/apache/spark/commit/79c2b29acf792c772866753913660fe69a42c279).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593707438
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604249875
 
 
   **[Test build #120401 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120401/testReport)** for PR 27728 at commit [`f732664`](https://github.com/apache/spark/commit/f732664630c6465ef8d4ec58dbc3965fcee82f3b).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593780092
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119193/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593776692
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119192/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gatorsmile commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r410831720
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ##########
 @@ -652,10 +652,19 @@ object DataSourceStrategy {
  */
 object PushableColumn {
   def unapply(e: Expression): Option[String] = {
-    def helper(e: Expression) = e match {
-      case a: Attribute => Some(a.name)
+    val nestedPredicatePushdownEnabled = SQLConf.get.nestedPredicatePushdownEnabled
+    import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+    def helper(e: Expression): Option[Seq[String]] = e match {
+      case a: Attribute =>
+        if (nestedPredicatePushdownEnabled || !a.name.contains(".")) {
 
 Review comment:
   Could you explain what this condition means?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604285635
 
 
   **[Test build #120410 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120410/testReport)** for PR 27728 at commit [`5fd97c0`](https://github.com/apache/spark/commit/5fd97c0a90eb1885a93fffb9d04a262b35f62bc3).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594175748
 
 
   **[Test build #119253 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119253/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-602930360
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24938/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398732474
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -502,33 +617,39 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
       (false, DecimalType.MAX_PRECISION) // binaryWriterUsingUnscaledBytes
     ).foreach { case (legacyFormat, precision) =>
       withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> legacyFormat.toString) {
-        val schema = StructType.fromDDL(s"a decimal($precision, 2)")
         val rdd =
           spark.sparkContext.parallelize((1 to 4).map(i => Row(new java.math.BigDecimal(i))))
-        val dataFrame = spark.createDataFrame(rdd, schema)
-        testDecimalPushDown(dataFrame) { implicit df =>
 
 Review comment:
   It's already removed here in this PR, https://github.com/apache/spark/pull/27728/files#diff-43b427b8b0b4b9d8dd7e4367c0526f83L157

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r390850480
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +134,10 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  private lazy val catalystSqlParser = new CatalystSqlParser(SQLConf.get)
+
+  def unquote(name: String): Seq[String] = {
 
 Review comment:
   it's not really unquote. How about `parseColumnPath`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-601974506
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593591634
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594215247
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594977415
 
 
   **[Test build #119350 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119350/testReport)** for PR 27728 at commit [`78b1a71`](https://github.com/apache/spark/commit/78b1a71b0dba02dd8b33d07eefab5fb099cfba8b).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387213594
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -128,37 +134,47 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     val ts3 = data(2)
     val ts4 = data(3)
 
-    withParquetDataFrame(data.map(i => Tuple1(i))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], data.map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 === ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 <=> ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 =!= ts1, classOf[NotEq[_]],
-        Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 < ts2, classOf[Lt[_]], ts1)
-      checkFilterPredicate('_1 > ts1, classOf[Gt[_]], Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-      checkFilterPredicate('_1 <= ts1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate('_1 >= ts4, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(Literal(ts1) === '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts1) <=> '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts2) > '_1, classOf[Lt[_]], ts1)
-      checkFilterPredicate(Literal(ts3) < '_1, classOf[Gt[_]], ts4)
-      checkFilterPredicate(Literal(ts1) >= '_1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate(Literal(ts4) <= '_1, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(!('_1 < ts4), classOf[GtEq[_]], ts4)
-      checkFilterPredicate('_1 < ts2 || '_1 > ts3, classOf[Operators.Or], Seq(Row(ts1), Row(ts4)))
-    }
-  }
-
-  private def testDecimalPushDown(data: DataFrame)(f: DataFrame => Unit): Unit = {
-    withTempPath { file =>
-      data.write.parquet(file.getCanonicalPath)
-      readParquetFile(file.toString)(f)
-    }
+    Seq(
+      (
+        spark.sqlContext.createDataFrame(data.map(x => ColA(Some(x)))),
 
 Review comment:
   Shall we use `spark` instead of `spark.sqlContext`?
   `SQLContext.createDataFrame` is an alias for `SparkSession.createDataFrame`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387323546
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 ##########
 @@ -49,15 +49,34 @@ class ParquetFilters(
     pushDownInFilterThreshold: Int,
     caseSensitive: Boolean) {
   // A map which contains parquet field name and data type, if predicate push down applies.
-  private val nameToParquetField : Map[String, ParquetField] = {
-    // Here we don't flatten the fields in the nested schema but just look up through
-    // root fields. Currently, accessing to nested fields does not push down filters
-    // and it does not support to create filters for them.
-    val primitiveFields =
-    schema.getFields.asScala.filter(_.isPrimitive).map(_.asPrimitiveType()).map { f =>
-      f.getName -> ParquetField(f.getName,
-        ParquetSchemaType(f.getOriginalType,
-          f.getPrimitiveTypeName, f.getTypeLength, f.getDecimalMetadata))
+  // The keys are the column names. For nested column, `dot` will be used as a separator.
+  // For column name that contains `dot`, backquote will be used.
+  // See `org.apache.spark.sql.connector.catalog.quote` for implementation details.
+  private val nameToParquetField : Map[String, ParquetPrimitiveField] = {
+    // Recursively traverse the parquet schema to get primitive fields that can be pushed-down.
+    // `parentFieldNames` is used to keep track of the current nested level when traversing.
+    def getPrimitiveFields(
+        fields: Seq[Type],
+        parentFieldNames: Array[String] = Array.empty): Seq[ParquetPrimitiveField] = {
+      fields.flatMap {
+        case p: PrimitiveType =>
+          Some(ParquetPrimitiveField(fieldNames = parentFieldNames :+ p.getName,
+            fieldType = ParquetSchemaType(p.getOriginalType,
+              p.getPrimitiveTypeName, p.getTypeLength, p.getDecimalMetadata)))
+        // Note that when g is a `Struct`, `g.getOriginalType` is `null`.
+        // When g is a `Map`, `g.getOriginalType` is `MAP`.
+        // When g is a `List`, `g.getOriginalType` is `LIST`.
+        case g: GroupType if g.getOriginalType == null =>
+          getPrimitiveFields(g.getFields.asScala, parentFieldNames :+ g.getName)
+        // Parquet only supports push-down for primitive types; as a result, Map and List types
+        // are removed.
+        case _ => None
+      }
+    }
+
+    val primitiveFields = getPrimitiveFields(schema.getFields.asScala).map { field =>
+      import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
 
 Review comment:
   this import contains implicit conversion, so I want to limit the scope.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-601920221
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24831/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594175748
 
 
   **[Test build #119253 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119253/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604329196
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r397654235
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -121,43 +121,81 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df)
   }
 
+  /**
+   * Takes single level `inputDF` dataframe to generate multi-level nested
+   * dataframes as new test data.
+   */
+  private def withNestedDataFrame(inputDF: DataFrame)
+      (testCases: (DataFrame, String, Any => Any) => Unit): Unit = {
 
 Review comment:
   `testCases` -> `runTest`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594256314
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604278610
 
 
   Retest this please.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592770419
 
 
   **[Test build #119104 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119104/testReport)** for PR 27728 at commit [`6972b54`](https://github.com/apache/spark/commit/6972b54a54793d65ea8b647347f6642c03b36352).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon edited a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon edited a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-607020100
 
 
   It's true that it can depends on the nature of data or workloads of course but I think it makes sense to at least have a benchmark though, including the (roughly) best and worst cases.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593591646
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23925/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592770419
 
 
   **[Test build #119104 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119104/testReport)** for PR 27728 at commit [`6972b54`](https://github.com/apache/spark/commit/6972b54a54793d65ea8b647347f6642c03b36352).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603852393
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592302027
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592292919
 
 
   **[Test build #119061 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119061/testReport)** for PR 27728 at commit [`4b27c82`](https://github.com/apache/spark/commit/4b27c8291046f6690761f35d19fac5e9fcbc7041).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597349202
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594176377
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23993/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594175454
 
 
   Retest this please.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592697610
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23845/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604578973
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592779547
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23848/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r397657185
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -121,43 +121,81 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df)
   }
 
+  /**
+   * Takes single level `inputDF` dataframe to generate multi-level nested
+   * dataframes as new test data.
+   */
+  private def withNestedDataFrame(inputDF: DataFrame)
+      (testCases: (DataFrame, String, Any => Any) => Unit): Unit = {
+    val df = inputDF.toDF("temp")
+    Seq(
+      (
+        df.withColumnRenamed("temp", "a"),
+        "a", // zero nesting
+        (x: Any) => x),
+      (
+        df.withColumn("a", struct(df("temp") as "b")).drop("temp"),
+        "a.b", // one level nesting
+        (x: Any) => Row(x)),
+      (
+        df.withColumn("a", struct(struct(df("temp") as "c") as "b")).drop("temp"),
+        "a.b.c", // two level nesting
+        (x: Any) => Row(Row(x))
+      ),
+      (
+        df.withColumnRenamed("temp", "a.b"),
+        "`a.b`", // zero nesting with column name containing `dots`
+        (x: Any) => x
+      ),
+      (
+        df.withColumn("a.b", struct(df("temp") as "c.d") ).drop("temp"),
+        "`a.b`.`c.d`", // one level nesting with column names containing `dots`
+        (x: Any) => Row(x)
+      )
+    ).foreach { case (df, pushDownColName, resultTransFun) =>
+      testCases(df, pushDownColName, resultTransFun)
+    }
+  }
+
   private def testTimestampPushdown(data: Seq[Timestamp]): Unit = {
     assert(data.size === 4)
     val ts1 = data.head
     val ts2 = data(1)
     val ts3 = data(2)
     val ts4 = data(3)
 
-    withParquetDataFrame(data.map(i => Tuple1(i))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], data.map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 === ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 <=> ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 =!= ts1, classOf[NotEq[_]],
-        Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 < ts2, classOf[Lt[_]], ts1)
-      checkFilterPredicate('_1 > ts1, classOf[Gt[_]], Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-      checkFilterPredicate('_1 <= ts1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate('_1 >= ts4, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(Literal(ts1) === '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts1) <=> '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts2) > '_1, classOf[Lt[_]], ts1)
-      checkFilterPredicate(Literal(ts3) < '_1, classOf[Gt[_]], ts4)
-      checkFilterPredicate(Literal(ts1) >= '_1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate(Literal(ts4) <= '_1, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(!('_1 < ts4), classOf[GtEq[_]], ts4)
-      checkFilterPredicate('_1 < ts2 || '_1 > ts3, classOf[Operators.Or], Seq(Row(ts1), Row(ts4)))
-    }
-  }
-
-  private def testDecimalPushDown(data: DataFrame)(f: DataFrame => Unit): Unit = {
-    withTempPath { file =>
-      data.write.parquet(file.getCanonicalPath)
-      readParquetFile(file.toString)(f)
+    import testImplicits._
+    withNestedDataFrame(data.toDF()) { case (df, pushDownColName, resultFun) =>
+      withParquetDataFrame(df) { implicit df =>
 
 Review comment:
   can we avoid name conflicts? there are two `df`s here. Shall we rename the first `df` to `inputDf`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-604224794
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594338740
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592248478
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23805/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r386668161
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +134,8 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  def unquote(name: String): Seq[String] = {
+    new CatalystSqlParser(SQLConf.get).parseMultipartIdentifier(name)
 
 Review comment:
   This shouldn't create a new parser each time it is used. Can you create a private field for the parser?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594113458
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592811208
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119104/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-605390464
 
 
   Ping @MaxGekk , are you interested in working on ORC version of it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594215247
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] viirya commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
viirya commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387266696
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 ##########
 @@ -75,13 +94,13 @@ class ParquetFilters(
   }
 
   /**
-   * Holds a single field information stored in the underlying parquet file.
+   * Holds a single primitive field information stored in the underlying parquet file.
    *
-   * @param fieldName field name in parquet file
+   * @param fieldNames field names in parquet file
 
 Review comment:
   I think It still indicates a single field name? Though it could contain multiple identifiers.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592727487
 
 
   **[Test build #119102 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119102/testReport)** for PR 27728 at commit [`0ae261d`](https://github.com/apache/spark/commit/0ae261d58cc1e967e499f4b9481a143c6234e17b).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592248472
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385982443
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala
 ##########
 @@ -466,13 +485,8 @@ class ParquetFilters(
     case _ => false
   }
 
-  // Parquet does not allow dots in the column name because dots are used as a column path
-  // delimiter. Since Parquet 1.8.2 (PARQUET-389), Parquet accepts the filter predicates
-  // with missing columns. The incorrect results could be got from Parquet when we push down
-  // filters for the column having dots in the names. Thus, we do not push down such filters.
-  // See SPARK-20364.
   private def canMakeFilterOn(name: String, value: Any): Boolean = {
-    nameToParquetField.contains(name) && !name.contains(".") && valueCanMakeFilterOn(name, value)
+    nameToParquetField.contains(name) && valueCanMakeFilterOn(name, value)
 
 Review comment:
   No. we don't have to. Should work.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592797695
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23852/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592806995
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119107/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593719228
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387216050
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala
 ##########
 @@ -63,13 +63,38 @@ private[sql] trait ParquetTest extends FileBasedDataSourceTest {
       (f: String => Unit): Unit = withDataSourceFile(data)(f)
 
   /**
-   * Writes `data` to a Parquet file and reads it back as a [[DataFrame]],
+   * Writes `data` objects to a Parquet file and reads it back as a [[DataFrame]],
    * which is then passed to `f`. The Parquet file will be deleted after `f` returns.
    */
-  protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag]
+  protected def withParquetDFfromObjs[T <: Product: ClassTag: TypeTag]
 
 Review comment:
   Can we have this renaming in a separate `TEST` PR first?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594338745
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119266/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387213664
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -128,37 +134,47 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     val ts3 = data(2)
     val ts4 = data(3)
 
-    withParquetDataFrame(data.map(i => Tuple1(i))) { implicit df =>
-      checkFilterPredicate('_1.isNull, classOf[Eq[_]], Seq.empty[Row])
-      checkFilterPredicate('_1.isNotNull, classOf[NotEq[_]], data.map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 === ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 <=> ts1, classOf[Eq[_]], ts1)
-      checkFilterPredicate('_1 =!= ts1, classOf[NotEq[_]],
-        Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-
-      checkFilterPredicate('_1 < ts2, classOf[Lt[_]], ts1)
-      checkFilterPredicate('_1 > ts1, classOf[Gt[_]], Seq(ts2, ts3, ts4).map(i => Row.apply(i)))
-      checkFilterPredicate('_1 <= ts1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate('_1 >= ts4, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(Literal(ts1) === '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts1) <=> '_1, classOf[Eq[_]], ts1)
-      checkFilterPredicate(Literal(ts2) > '_1, classOf[Lt[_]], ts1)
-      checkFilterPredicate(Literal(ts3) < '_1, classOf[Gt[_]], ts4)
-      checkFilterPredicate(Literal(ts1) >= '_1, classOf[LtEq[_]], ts1)
-      checkFilterPredicate(Literal(ts4) <= '_1, classOf[GtEq[_]], ts4)
-
-      checkFilterPredicate(!('_1 < ts4), classOf[GtEq[_]], ts4)
-      checkFilterPredicate('_1 < ts2 || '_1 > ts3, classOf[Operators.Or], Seq(Row(ts1), Row(ts4)))
-    }
-  }
-
-  private def testDecimalPushDown(data: DataFrame)(f: DataFrame => Unit): Unit = {
-    withTempPath { file =>
-      data.write.parquet(file.getCanonicalPath)
-      readParquetFile(file.toString)(f)
-    }
+    Seq(
+      (
+        spark.sqlContext.createDataFrame(data.map(x => ColA(Some(x)))),
+        "a", // zero nesting
+        (x: Any) => x),
+      (
+        spark.sqlContext.createDataFrame(data.map(x => ColA(Some(ColB(Some(x)))))),
 
 Review comment:
   ditto.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592865781
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119108/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593780092
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119193/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gengliangwang commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385929469
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -40,6 +40,14 @@ sealed abstract class Filter {
     case f: Filter => f.references
     case _ => Array.empty
   }
+
+  /**
+   * If any of the references of this filter contains nested field
+   * @since 3.2.0
+   */
+  def containNestedField: Boolean = {
+    this.references.exists(_.contains("."))
 
 Review comment:
   How about pass the schema or top-level output attributes as a parameter, and judge if the filter is of a nested column.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594164190
 
 
   **[Test build #119248 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119248/testReport)** for PR 27728 at commit [`77ea177`](https://github.com/apache/spark/commit/77ea177985516e05bf89ebbbb3c05a9c87050583).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592248105
 
 
   **[Test build #119060 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119060/testReport)** for PR 27728 at commit [`3a2fb39`](https://github.com/apache/spark/commit/3a2fb390975b97a60005d4df275bfddaf5d07a6d).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594977415
 
 
   **[Test build #119350 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119350/testReport)** for PR 27728 at commit [`78b1a71`](https://github.com/apache/spark/commit/78b1a71b0dba02dd8b33d07eefab5fb099cfba8b).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r385962101
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala
 ##########
 @@ -22,69 +22,119 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.PlanTest
 import org.apache.spark.sql.sources
 import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
 
 class DataSourceStrategySuite extends PlanTest with SharedSparkSession {
 
-  test("translate simple expression") {
-    val attrInt = 'cint.int
-    val attrStr = 'cstr.string
-
-    testTranslateFilter(EqualTo(attrInt, 1), Some(sources.EqualTo("cint", 1)))
-    testTranslateFilter(EqualTo(1, attrInt), Some(sources.EqualTo("cint", 1)))
+  val attrInts = Seq(
+    'cint.int,
+    Symbol("c.int").int,
+    GetStructField('a.struct(StructType(
+      StructField("cstr", StringType, nullable = true) ::
+        StructField("cint", IntegerType, nullable = true) :: Nil)), 1, None),
+    GetStructField('a.struct(StructType(
+      StructField("c.int", IntegerType, nullable = true) ::
+        StructField("cstr", StringType, nullable = true) :: Nil)), 0, None),
+    GetStructField(Symbol("a.b").struct(StructType(
+      StructField("cstr1", StringType, nullable = true) ::
+        StructField("cstr2", StringType, nullable = true) ::
+        StructField("cint", IntegerType, nullable = true) :: Nil)), 2, None),
+    GetStructField(Symbol("a.b").struct(StructType(
+      StructField("c.int", IntegerType, nullable = true) :: Nil)), 0, None)
+  ).zip(Seq(
+    "cint",
+    "`c.int`", // single level column that contains `dot` in name
+    "a.cint", // two level nested column
+    "a.`c.int`", // two level nested column, and nested level contains `dot`
+    "`a.b`.cint", // two level nested column, and top level contains `dot`
+    "`a.b`.`c.int`" // two level nested column, and both levels contain `dot`
 
 Review comment:
   I added a test case for three level nested field.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387319351
 
 

 ##########
 File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ##########
 @@ -437,61 +437,74 @@ object DataSourceStrategy {
     }
   }
 
+  /**
+   * Find the column name of an expression that can be pushed down.
+   */
+  private[sql] def pushDownColName(e: Expression): Option[String] = {
+    import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper
+    def helper(e: Expression): Option[Seq[String]] = e match {
+      case a: Attribute => Some(Seq(a.name))
+      case s: GetStructField => helper(s.child).map(_ :+ s.childSchema(s.ordinal).name)
+      case _ => None
+    }
+    helper(e).map(_.quoted)
+  }
+
   private def translateLeafNodeFilter(predicate: Expression): Option[Filter] = predicate match {
-    case expressions.EqualTo(a: Attribute, Literal(v, t)) =>
-      Some(sources.EqualTo(a.name, convertToScala(v, t)))
-    case expressions.EqualTo(Literal(v, t), a: Attribute) =>
-      Some(sources.EqualTo(a.name, convertToScala(v, t)))
-
-    case expressions.EqualNullSafe(a: Attribute, Literal(v, t)) =>
-      Some(sources.EqualNullSafe(a.name, convertToScala(v, t)))
-    case expressions.EqualNullSafe(Literal(v, t), a: Attribute) =>
-      Some(sources.EqualNullSafe(a.name, convertToScala(v, t)))
-
-    case expressions.GreaterThan(a: Attribute, Literal(v, t)) =>
-      Some(sources.GreaterThan(a.name, convertToScala(v, t)))
-    case expressions.GreaterThan(Literal(v, t), a: Attribute) =>
-      Some(sources.LessThan(a.name, convertToScala(v, t)))
-
-    case expressions.LessThan(a: Attribute, Literal(v, t)) =>
-      Some(sources.LessThan(a.name, convertToScala(v, t)))
-    case expressions.LessThan(Literal(v, t), a: Attribute) =>
-      Some(sources.GreaterThan(a.name, convertToScala(v, t)))
-
-    case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, t)) =>
-      Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t)))
-    case expressions.GreaterThanOrEqual(Literal(v, t), a: Attribute) =>
-      Some(sources.LessThanOrEqual(a.name, convertToScala(v, t)))
-
-    case expressions.LessThanOrEqual(a: Attribute, Literal(v, t)) =>
-      Some(sources.LessThanOrEqual(a.name, convertToScala(v, t)))
-    case expressions.LessThanOrEqual(Literal(v, t), a: Attribute) =>
-      Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t)))
-
-    case expressions.InSet(a: Attribute, set) =>
-      val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType)
-      Some(sources.In(a.name, set.toArray.map(toScala)))
+    case expressions.EqualTo(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.EqualTo(_, convertToScala(v, t)))
+    case expressions.EqualTo(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.EqualTo(_, convertToScala(v, t)))
+
+    case expressions.EqualNullSafe(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.EqualNullSafe(_, convertToScala(v, t)))
+    case expressions.EqualNullSafe(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.EqualNullSafe(_, convertToScala(v, t)))
+
+    case expressions.GreaterThan(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.GreaterThan(_, convertToScala(v, t)))
+    case expressions.GreaterThan(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.LessThan(_, convertToScala(v, t)))
+
+    case expressions.LessThan(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.LessThan(_, convertToScala(v, t)))
+    case expressions.LessThan(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.GreaterThan(_, convertToScala(v, t)))
+
+    case expressions.GreaterThanOrEqual(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.GreaterThanOrEqual(_, convertToScala(v, t)))
+    case expressions.GreaterThanOrEqual(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.LessThanOrEqual(_, convertToScala(v, t)))
+
+    case expressions.LessThanOrEqual(e: Expression, Literal(v, t)) =>
+      pushDownColName(e).map(sources.LessThanOrEqual(_, convertToScala(v, t)))
+    case expressions.LessThanOrEqual(Literal(v, t), e: Expression) =>
+      pushDownColName(e).map(sources.GreaterThanOrEqual(_, convertToScala(v, t)))
+
+    case expressions.InSet(e: Expression, set) =>
+      val toScala = CatalystTypeConverters.createToScalaConverter(e.dataType)
+      pushDownColName(e).map(sources.In(_, set.toArray.map(toScala)))
 
     // Because we only convert In to InSet in Optimizer when there are more than certain
     // items. So it is possible we still get an In expression here that needs to be pushed
     // down.
-    case expressions.In(a: Attribute, list) if list.forall(_.isInstanceOf[Literal]) =>
+    case expressions.In(e: Expression, list) if list.forall(_.isInstanceOf[Literal]) =>
       val hSet = list.map(_.eval(EmptyRow))
-      val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType)
-      Some(sources.In(a.name, hSet.toArray.map(toScala)))
+      val toScala = CatalystTypeConverters.createToScalaConverter(e.dataType)
 
 Review comment:
   Done. thanks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-602932269
 
 
   **[Test build #120225 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120225/testReport)** for PR 27728 at commit [`22a7a00`](https://github.com/apache/spark/commit/22a7a0029943a696dc0f085972d8568473c24ec5).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398310277
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -121,43 +121,81 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df)
   }
 
+  /**
+   * Takes single level `inputDF` dataframe to generate multi-level nested
+   * dataframes as new test data.
+   */
+  private def withNestedDataFrame(inputDF: DataFrame)
 
 Review comment:
   Okay, this is not easy since one of the test case is like
   ```scala
           val dataFrame = spark.createDataFrame(rdd, StructType.fromDDL(s"a decimal($precision, 2)"))
           withNestedDataFrame(dataFrame) { case (inputDF, pushDownColName, resultFun) =>
             withParquetDataFrame(inputDF) { implicit df =>
               val decimalAttr: Expression = df(pushDownColName).expr
               assert(df(pushDownColName).expr.dataType === DecimalType(precision, 2))
   ```
   , and the dataframe can not be constructed directly from `withNestedDataFrame[T](data: Seq[T])`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-603638824
 
 
   **[Test build #120307 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120307/testReport)** for PR 27728 at commit [`44b310e`](https://github.com/apache/spark/commit/44b310e423250c4635943c8e66323f37f03ef75b).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593591062
 
 
   **[Test build #119185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119185/testReport)** for PR 27728 at commit [`af72b3b`](https://github.com/apache/spark/commit/af72b3b609b617b94d83217ffc07b96559335b79).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-596038591
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-597220544
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594164391
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119248/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592811205
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592254219
 
 
   **[Test build #119061 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119061/testReport)** for PR 27728 at commit [`4b27c82`](https://github.com/apache/spark/commit/4b27c8291046f6690761f35d19fac5e9fcbc7041).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592297066
 
 
   **[Test build #119059 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119059/testReport)** for PR 27728 at commit [`400aebe`](https://github.com/apache/spark/commit/400aebe9d2526609c0a107a93d849288cac20708).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592243369
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593591062
 
 
   **[Test build #119185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119185/testReport)** for PR 27728 at commit [`af72b3b`](https://github.com/apache/spark/commit/af72b3b609b617b94d83217ffc07b96559335b79).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gatorsmile commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
gatorsmile commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-606986453
 
 
   @dbtsai Great work! Could you show some perf number ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593719235
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23934/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-594251580
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24005/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r386594772
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Implicits.scala
 ##########
 @@ -132,4 +132,9 @@ private[sql] object CatalogV2Implicits {
       part
     }
   }
+
+  def unqote(input: String): Seq[String] = {
+    // TODO: Handle the column name contains `dot` properly
+    input.split('.')
 
 Review comment:
   You can copy how this is done in [`LogicalExpressions`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/expressions/expressions.scala#L32-L34).
   
   ```scala
   val parser = new CatalystSqlParser(SQLConf.get)
   val nameParts = parser.parseMultipartIdentifier(name)
   ```
   
   You can also use the `parseReference` method if you want to get a `FieldReference`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27728: [SPARK-17636][SQL] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-593681207
 
 
   **[Test build #119185 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119185/testReport)** for PR 27728 at commit [`af72b3b`](https://github.com/apache/spark/commit/af72b3b609b617b94d83217ffc07b96559335b79).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r392507441
 
 

 ##########
 File path: sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala
 ##########
 @@ -73,9 +74,11 @@ private[orc] object OrcFilters extends Logging {
     if (HiveUtils.isHive23) {
       DatasourceOrcFilters.createFilter(schema, filters).asInstanceOf[Option[SearchArgument]]
     } else {
-      val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap
+      val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap
+      // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed.
 
 Review comment:
   ORC already supports column name with `dots`. In fact, ORC API is using the same quotation method I purposed in this PR, but we add the quotation if the column name contains dot right before calling ORC filter APIs. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398538126
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -502,33 +617,39 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
       (false, DecimalType.MAX_PRECISION) // binaryWriterUsingUnscaledBytes
     ).foreach { case (legacyFormat, precision) =>
       withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> legacyFormat.toString) {
-        val schema = StructType.fromDDL(s"a decimal($precision, 2)")
         val rdd =
           spark.sparkContext.parallelize((1 to 4).map(i => Row(new java.math.BigDecimal(i))))
-        val dataFrame = spark.createDataFrame(rdd, schema)
-        testDecimalPushDown(dataFrame) { implicit df =>
 
 Review comment:
   We can remove the `testDecimalPushDown` method now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
dbtsai commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r398312781
 
 

 ##########
 File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala
 ##########
 @@ -121,43 +121,81 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared
     checkBinaryFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df)
   }
 
+  /**
+   * Takes single level `inputDF` dataframe to generate multi-level nested
+   * dataframes as new test data.
+   */
+  private def withNestedDataFrame(inputDF: DataFrame)
 
 Review comment:
   Instead, I add schema checking in the code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-595056224
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119353/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SPARK-31060][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r396838293
 
 

 ##########
 File path: sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala
 ##########
 @@ -40,12 +45,32 @@ sealed abstract class Filter {
     case f: Filter => f.references
     case _ => Array.empty
   }
+
+  /**
+   * List of columns that are referenced by this filter.
+   *
+   * @return each element is a column name as an array of string multi-identifier
+   * @since 3.0.0
+   */
+  def V2references: Array[Array[String]] = {
 
 Review comment:
   nit: `V2references` -> `v2references`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27728: [SPARK-17636][SQL] Nested Field Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#issuecomment-592697610
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23845/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #27728: [SPARK-25556][SPARK-17636][SPARK-31026][SQL][test-hive1.2] Nested Column Predicate Pushdown for Parquet
URL: https://github.com/apache/spark/pull/27728#discussion_r387396611
 
 

 ##########
 File path: sql/core/src/main/java/org/apache/parquet/filter2/predicate/NestedFilterApi.java
 ##########
 @@ -0,0 +1,57 @@
+/*
+ * 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.parquet.filter2.predicate;
+
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.filter2.predicate.Operators.BinaryColumn;
+import org.apache.parquet.filter2.predicate.Operators.BooleanColumn;
+import org.apache.parquet.filter2.predicate.Operators.DoubleColumn;
+import org.apache.parquet.filter2.predicate.Operators.FloatColumn;
+import org.apache.parquet.filter2.predicate.Operators.IntColumn;
+import org.apache.parquet.filter2.predicate.Operators.LongColumn;
+
+/**
+ * TODO: This is a temporary workaround; it is intended to be moved to Parquet project.
+ */
 
 Review comment:
   I am a bit confused about the decision made here. This way was't recommended before by @rdblue. Am I remembering wrongly, @rdblue?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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