You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by mallman <gi...@git.apache.org> on 2018/05/14 09:59:34 UTC

[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

GitHub user mallman opened a pull request:

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

    [SPARK-4502][SQL] Parquet nested column pruning - foundation

    (Link to Jira: https://issues.apache.org/jira/browse/SPARK-4502)
    
    _N.B. This is a restart of PR #16578 which includes everything in that PR except the aggregation and join schema pruning rules. Relevant review comments from that PR should be considered incorporated by reference. Please avoid duplication in review by reviewing that PR first. The summary below is an edited copy of the summary of the previous PR._
    
    ## What changes were proposed in this pull request?
    
    One of the hallmarks of a column-oriented data storage format is the ability to read data from a subset of columns, efficiently skipping reads from other columns. Spark has long had support for pruning unneeded top-level schema fields from the scan of a parquet file. For example, consider a table, `contacts`, backed by parquet with the following Spark SQL schema:
    
    ```
    root
     |-- name: struct
     |    |-- first: string
     |    |-- last: string
     |-- address: string
    ```
    
    Parquet stores this table's data in three physical columns: `name.first`, `name.last` and `address`. To answer the query
    
    ```SQL
    select address from contacts
    ```
    
    Spark will read only from the `address` column of parquet data. However, to answer the query
    
    ```SQL
    select name.first from contacts
    ```
    
    Spark will read `name.first` and `name.last` from parquet.
    
    This PR modifies Spark SQL to support a finer-grain of schema pruning. With this patch, Spark reads only the `name.first` column to answer the previous query.
    
    ### Implementation
    
    There are two main components of this patch. First, there is a `ParquetSchemaPruning` optimizer rule for gathering the required schema fields of a `PhysicalOperation` over a parquet file, constructing a new schema based on those required fields and rewriting the plan in terms of that pruned schema. The pruned schema fields are pushed down to the parquet requested read schema. `ParquetSchemaPruning` uses a new `ProjectionOverSchema` extractor for rewriting a catalyst expression in terms of a pruned schema.
    
    Second, the `ParquetRowConverter` has been patched to ensure the ordinals of the parquet columns read are correct for the pruned schema. `ParquetReadSupport` has been patched to address a compatibility mismatch between Spark's built in vectorized reader and the parquet-mr library's reader.
    
    ### Limitation
    
    Among the complex Spark SQL data types, this patch supports parquet column pruning of nested sequences of struct fields only.
    
    ## How was this patch tested?
    
    Care has been taken to ensure correctness and prevent regressions. A more advanced version of this patch incorporating optimizations for rewriting queries involving aggregations and joins has been running on a production Spark cluster at VideoAmp for several years. In that time, one bug was found and fixed early on, and we added a regression test for that bug.
    
    We forward-ported this patch to Spark master in June 2016 and have been running this patch against Spark 2.x branches on ad-hoc clusters since then.

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

    $ git pull https://github.com/VideoAmp/spark-public spark-4502-parquet_column_pruning-foundation

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

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

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

    This closes #21320
    
----
commit 9e301b37bb5863ef5fad8ec15f1d8f3d4b5e6a6f
Author: Michael Allman <mi...@...>
Date:   2016-06-24T17:21:24Z

    [SPARK-4502][SQL] Parquet nested column pruning

----


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199364935
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -71,9 +80,22 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
           StructType.fromString(schemaString)
         }
     
    -    val parquetRequestedSchema =
    +    val clippedParquetSchema =
           ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema)
     
    +    val parquetRequestedSchema = if (parquetMrCompatibility) {
    --- End diff --
    
    This is only for nested schema pruning? Turn this off when `spark.sql.nestedSchemaPruning.enabled` is off?


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94224 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94224/testReport)** for PR 21320 at commit [`a87b589`](https://github.com/apache/spark/commit/a87b589c70af69d25424b03b52e29165ab4e0b5f).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205022799
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ---
    @@ -0,0 +1,156 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import java.io.File
    +
    +import org.apache.spark.sql.{QueryTest, Row}
    +import org.apache.spark.sql.execution.FileSchemaPruningTest
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class ParquetSchemaPruningSuite
    +    extends QueryTest
    +    with ParquetTest
    +    with FileSchemaPruningTest
    +    with SharedSQLContext {
    +  case class FullName(first: String, middle: String, last: String)
    +  case class Contact(name: FullName, address: String, pets: Int, friends: Array[FullName] = Array(),
    +    relatives: Map[String, FullName] = Map())
    +
    +  val contacts =
    +    Contact(FullName("Jane", "X.", "Doe"), "123 Main Street", 1) ::
    +    Contact(FullName("John", "Y.", "Doe"), "321 Wall Street", 3) :: Nil
    +
    +  case class Name(first: String, last: String)
    +  case class BriefContact(name: Name, address: String)
    +
    +  val briefContacts =
    +    BriefContact(Name("Janet", "Jones"), "567 Maple Drive") ::
    +    BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil
    +
    +  case class ContactWithDataPartitionColumn(name: FullName, address: String, pets: Int,
    +    friends: Array[FullName] = Array(), relatives: Map[String, FullName] = Map(), p: Int)
    +
    +  case class BriefContactWithDataPartitionColumn(name: Name, address: String, p: Int)
    +
    +  val contactsWithDataPartitionColumn =
    +    contacts.map { case Contact(name, address, pets, friends, relatives) =>
    +      ContactWithDataPartitionColumn(name, address, pets, friends, relatives, 1) }
    +  val briefContactsWithDataPartitionColumn =
    +    briefContacts.map { case BriefContact(name: Name, address: String) =>
    +      BriefContactWithDataPartitionColumn(name, address, 2) }
    +
    +  testSchemaPruning("select a single complex field") {
    +    val query = sql("select name.middle from contacts")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil)
    +  }
    +
    +  testSchemaPruning("select a single complex field and the partition column") {
    +    val query = sql("select name.middle, p from contacts")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil)
    +  }
    +
    +  ignore("partial schema intersection - select missing subfield") {
    +    val query = sql("select name.middle, address from contacts where p=2")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>,address:string>")
    +    checkAnswer(query,
    +      Row(null, "567 Maple Drive") ::
    +      Row(null, "6242 Ash Street") :: Nil)
    +  }
    +
    +  testSchemaPruning("no unnecessary schema pruning") {
    +    val query =
    +      sql("select name.last, name.middle, name.first, relatives[''].last, " +
    +        "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " +
    +        "friends[0].first, pets, address from contacts where p=2")
    +    // We've selected every field in the schema. Therefore, no schema pruning should be performed.
    +    // We check this by asserting that the scanned schema of the query is identical to the schema
    +    // of the contacts relation, even though the fields are selected in different orders.
    +    checkScanSchemata(query,
    +      "struct<name:struct<first:string,middle:string,last:string>,address:string,pets:int," +
    +      "friends:array<struct<first:string,middle:string,last:string>>," +
    +      "relatives:map<string,struct<first:string,middle:string,last:string>>>")
    +    checkAnswer(query,
    +      Row("Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") ::
    +      Row("Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") ::
    +      Nil)
    +  }
    +
    +  testSchemaPruning("empty schema intersection") {
    +    val query = sql("select name.middle from contacts where p=2")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query,
    +      Row(null) :: Row(null) :: Nil)
    +  }
    +
    +  private def testSchemaPruning(testName: String)(testThunk: => Unit) {
    +    withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") {
    +      test(s"Spark vectorized reader - without partition data column - $testName") {
    +        withContacts(testThunk)
    +      }
    +      test(s"Spark vectorized reader - with partition data column - $testName") {
    +        withContactsWithDataPartitionColumn(testThunk)
    +      }
    +    }
    +
    +    withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") {
    +      test(s"Parquet-mr reader - without partition data column - $testName") {
    --- End diff --
    
    I think the "native parquet" term is ambiguous, and parquet-mr is not.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by DaimonPl <gi...@git.apache.org>.
Github user DaimonPl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r195704995
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -1288,8 +1288,18 @@ object SQLConf {
             "issues. Turn on this config to insert a local sort before actually doing repartition " +
             "to generate consistent repartition results. The performance of repartition() may go " +
             "down since we insert extra local sort before it.")
    +        .booleanConf
    +        .createWithDefault(true)
    +
    +  val NESTED_SCHEMA_PRUNING_ENABLED =
    +    buildConf("spark.sql.nestedSchemaPruning.enabled")
    +      .internal()
    +      .doc("Prune nested fields from a logical relation's output which are unnecessary in " +
    +        "satisfying a query. This optimization allows columnar file format readers to avoid " +
    +        "reading unnecessary nested column data. Currently Parquet is the only data source that " +
    +        "implements this optimization.")
           .booleanConf
    -      .createWithDefault(true)
    +      .createWithDefault(false)
    --- End diff --
    
    how about enabling it as default? there should be enough time to find any unexpected problems with 2.4.0
    
    + nested column pruning would be enabled during for all other automatic tests


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > gentle ping @mallman since the code freeze is close
    
    Outside of my primary occupation, my top priority on this PR right now is investigating 
    https://github.com/apache/spark/pull/21320#issuecomment-396498487. I don't think I'm going to get a test file from the OP, so I'm going to try to reproduce the issue on my own.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @gatorsmile How does this look?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91442/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > After more than two years of off and on review, discussion/debate, nitpicking, commits, steps forward and backwards, to have someone swoop in at this time with a new raft of nitpicking and stylistic issues that set the review back again further is beyond maddening. 
    
    I think that's primarily because the change looks incomplete but the feature itself sounds good to have. I think that's why people try to take a look a lot.
    
    Stepping forward and backwards is bad. That's why I am sticking with this PR to get this change in and help you address other people's comments and prevent such forward and backward.
    
    Stylistic issues are virtually based upon https://github.com/databricks/scala-style-guide .
    
    Nitpicking from me is basically from referring other codes or PRs in Spark, or other committer's preference so that we can get through this. I guess nits are still good to fix if you happen to push more changes. I guess it would take few seconds to address them. If that's not, please ignore my nit or minor comments. They don't block the PR usually.
    
    For clarification, few comments mentioned in https://github.com/apache/spark/pull/21320#issuecomment-407714036 are pretty reject comments in general in other PRs too.
    
    > Contributing to this PR is a tax on what is completely voluntary, unpaid time.
    
    FWIW, all my works have been unpaid and completely voluntary to me more than 3 years in the past except the recent half 6 months (which basically means until I became a committer). To be honest, I believe I still work on Spark like when I worked individually before.
    
    > I have no professional responsibility to this effort. Maybe it's better off done by someone who does.
    
    I completely agree. There should be no professional responsibility like a task to do in an open source in general. I think no one has that professional responsibility to take this and here we should be transparent on this. If anyone interested in this finds that you want someone else to take over, this might be taken over _voluntarily_ with a comment saying I want to take over this.
    
    I might cc some people who might be interested in this in order to inform them but it doesn't mean I hand it off to someone else.
    
    I am sorry if you felt I am pushing or rushing you - was trying to get this change in since people find it's a good feature to have. That's why I prioritized this and stick to this PR.
    



---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > the window of opportunity to review syntax and style in this PR closed long ago.
    
    Why/when is this window closed? Who closed that?
    
    This project claims a set of specific styles and they should better be fixed if more changes should be fixed. You stated that you refuse to deal with non-style comments too, right? For example, enabling the default value to see if any test is broken, or getting rid of weird hacks with `withSQL` in the tests. These are not style nits. Even, nits should better be addressed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Another way is you rebase @ajacques's commit here and he push some changes into here since you refuse to address some comments here. I still don't understand why refuse this though. if that's only way to address them, we can go this way too.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    retest this please


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94915 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94915/testReport)** for PR 21320 at commit [`1573ae8`](https://github.com/apache/spark/commit/1573ae888d651a51e0d60683117714fba7c55fb0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93390/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r204206252
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ---
    @@ -0,0 +1,156 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import java.io.File
    +
    +import org.apache.spark.sql.{QueryTest, Row}
    +import org.apache.spark.sql.execution.FileSchemaPruningTest
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class ParquetSchemaPruningSuite
    +    extends QueryTest
    +    with ParquetTest
    +    with FileSchemaPruningTest
    +    with SharedSQLContext {
    +  case class FullName(first: String, middle: String, last: String)
    +  case class Contact(name: FullName, address: String, pets: Int, friends: Array[FullName] = Array(),
    +    relatives: Map[String, FullName] = Map())
    +
    +  val contacts =
    +    Contact(FullName("Jane", "X.", "Doe"), "123 Main Street", 1) ::
    +    Contact(FullName("John", "Y.", "Doe"), "321 Wall Street", 3) :: Nil
    +
    +  case class Name(first: String, last: String)
    +  case class BriefContact(name: Name, address: String)
    +
    +  val briefContacts =
    +    BriefContact(Name("Janet", "Jones"), "567 Maple Drive") ::
    +    BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil
    +
    +  case class ContactWithDataPartitionColumn(name: FullName, address: String, pets: Int,
    +    friends: Array[FullName] = Array(), relatives: Map[String, FullName] = Map(), p: Int)
    +
    +  case class BriefContactWithDataPartitionColumn(name: Name, address: String, p: Int)
    +
    +  val contactsWithDataPartitionColumn =
    +    contacts.map { case Contact(name, address, pets, friends, relatives) =>
    +      ContactWithDataPartitionColumn(name, address, pets, friends, relatives, 1) }
    +  val briefContactsWithDataPartitionColumn =
    +    briefContacts.map { case BriefContact(name: Name, address: String) =>
    +      BriefContactWithDataPartitionColumn(name, address, 2) }
    +
    +  testSchemaPruning("select a single complex field") {
    +    val query = sql("select name.middle from contacts")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil)
    +  }
    +
    +  testSchemaPruning("select a single complex field and the partition column") {
    +    val query = sql("select name.middle, p from contacts")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil)
    +  }
    +
    +  testSchemaPruning("partial schema intersection - select missing subfield") {
    +    val query = sql("select name.middle, address from contacts where p=2")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>,address:string>")
    +    checkAnswer(query,
    +      Row(null, "567 Maple Drive") ::
    +      Row(null, "6242 Ash Street") :: Nil)
    +  }
    +
    +  testSchemaPruning("no unnecessary schema pruning") {
    +    val query =
    +      sql("select name.last, name.middle, name.first, relatives[''].last, " +
    +        "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " +
    +        "friends[0].first, pets, address from contacts where p=2")
    +    // We've selected every field in the schema. Therefore, no schema pruning should be performed.
    +    // We check this by asserting that the scanned schema of the query is identical to the schema
    +    // of the contacts relation, even though the fields are selected in different orders.
    +    checkScanSchemata(query,
    +      "struct<name:struct<first:string,middle:string,last:string>,address:string,pets:int," +
    +      "friends:array<struct<first:string,middle:string,last:string>>," +
    +      "relatives:map<string,struct<first:string,middle:string,last:string>>>")
    +    checkAnswer(query,
    +      Row("Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") ::
    +      Row("Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") ::
    +      Nil)
    +  }
    +
    +  testSchemaPruning("empty schema intersection") {
    +    val query = sql("select name.middle from contacts where p=2")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query,
    +      Row(null) :: Row(null) :: Nil)
    +  }
    +
    +  private def testSchemaPruning(testName: String)(testThunk: => Unit) {
    +    withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") {
    --- End diff --
    
    Yup.
    
    If I am not mistaken, Spark has its's own implementation for vectorized reader alone, and we have a normal Parquet reader otherwise.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by dbtsai <gi...@git.apache.org>.
Github user dbtsai commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    cc @viirya 
    
    If we select a nested field and a top level field, the schema pruning will fail. Here is the reproducible test,
    ```scala
      testSchemaPruning("select a single complex field and a top level field") {
        val query = sql("select * from contacts")
          .select("name.middle", "address")
        query.explain(true)
        query.printSchema()
        query.show()
        checkScan(query, "struct<name:struct<middle:string>,address:string>")
      }
    ```
    
    and the exception is
    
    ```
    23:16:05.864 ERROR org.apache.spark.executor.Executor: Exception in task 1.0 in stage 3.0 (TID 6)
    org.apache.spark.sql.execution.QueryExecutionException: Encounter error while reading parquet files. One possible cause: Parquet column cannot be converted in the corresponding files. Details: 
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:193)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
    	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
    	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:674)
    	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:255)
    	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
    	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
    	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
    	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:325)
    	at org.apache.spark.rdd.RDD.iterator(RDD.scala:289)
    	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
    	at org.apache.spark.scheduler.Task.run(Task.scala:121)
    	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$11.apply(Executor.scala:419)
    	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
    	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:425)
    	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    	at java.lang.Thread.run(Thread.java:748)
    Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in block -1 in file file:/private/var/folders/pr/4q3b9vkx36lbygjr5jhfmjcw0000gn/T/spark-a4fff68d-d51a-4c79-aa18-54cfd7f81a75/contacts/p=2/part-00000-8a4d9396-7be3-4fed-a55a-5580684ebda6-c000.snappy.parquet
    	at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
    	at org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
    	at org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)
    	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:181)
    	... 19 more
    Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
    	at java.util.ArrayList.rangeCheck(ArrayList.java:657)
    	at java.util.ArrayList.get(ArrayList.java:433)
    	at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
    	at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
    	at org.apache.parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:97)
    	at org.apache.parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:92)
    	at org.apache.parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:278)
    	at org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:147)
    	at org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:109)
    	at org.apache.parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:165)
    	at org.apache.parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:109)
    	at org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:137)
    	at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:222)
    	... 24 more
    23:16:05.896 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in stage 3.0 (TID 6, localhost, executor driver): org.apache.spark.sql.execution.QueryExecutionException: Encounter error while reading parquet files. One possible cause: Parquet column cannot be converted in the corresponding files. Details: 
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:193)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
    	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
    	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$11$$anon$1.hasNext(WholeStageCodegenExec.scala:674)
    	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:255)
    	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:247)
    	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
    	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:850)
    	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
    	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:325)
    	at org.apache.spark.rdd.RDD.iterator(RDD.scala:289)
    	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
    	at org.apache.spark.scheduler.Task.run(Task.scala:121)
    	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$11.apply(Executor.scala:419)
    	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
    	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:425)
    	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    	at java.lang.Thread.run(Thread.java:748)
    Caused by: org.apache.parquet.io.ParquetDecodingException: Can not read value at 0 in block -1 in file file:/private/var/folders/pr/4q3b9vkx36lbygjr5jhfmjcw0000gn/T/spark-a4fff68d-d51a-4c79-aa18-54cfd7f81a75/contacts/p=2/part-00000-8a4d9396-7be3-4fed-a55a-5580684ebda6-c000.snappy.parquet
    	at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
    	at org.apache.parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:207)
    	at org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)
    	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:409)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:101)
    	at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:181)
    	... 19 more
    Caused by: java.lang.IndexOutOfBoundsException: Index: 0, Size: 0
    	at java.util.ArrayList.rangeCheck(ArrayList.java:657)
    	at java.util.ArrayList.get(ArrayList.java:433)
    	at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
    	at org.apache.parquet.io.GroupColumnIO.getFirst(GroupColumnIO.java:99)
    	at org.apache.parquet.io.PrimitiveColumnIO.getFirst(PrimitiveColumnIO.java:97)
    	at org.apache.parquet.io.PrimitiveColumnIO.isFirst(PrimitiveColumnIO.java:92)
    	at org.apache.parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:278)
    	at org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:147)
    	at org.apache.parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:109)
    	at org.apache.parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:165)
    	at org.apache.parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:109)
    	at org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:137)
    	at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:222)
    	... 24 more
    ```


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199368095
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -47,16 +47,25 @@ import org.apache.spark.sql.types._
      *
      * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]]
      * to [[prepareForRead()]], but use a private `var` for simplicity.
    + *
    + * @param parquetMrCompatibility support reading with parquet-mr or Spark's built-in Parquet reader
      */
    -private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
    +private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone],
    +    parquetMrCompatibility: Boolean)
         extends ReadSupport[UnsafeRow] with Logging {
       private var catalystRequestedSchema: StructType = _
     
    +  /**
    +   * Construct a [[ParquetReadSupport]] with [[convertTz]] set to [[None]] and
    +   * [[parquetMrCompatibility]] set to [[false]].
    +   *
    +   * We need a zero-arg constructor for SpecificParquetRecordReaderBase.  But that is only
    +   * used in the vectorized reader, where we get the convertTz value directly, and the value here
    +   * is ignored. Further, we set [[parquetMrCompatibility]] to [[false]] as this constructor is only
    +   * called by the Spark reader.
    --- End diff --
    
    Based on this comment, we have no idea why this is set to false. 


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Few comments like #21320 (comment) or ^ are not minor or nits. I leave hard -1 if they are not addressed.
    
    I'm sorry to say I'm very close to hanging up this PR. I put a lot of care, time and effort into my work. After more than two years of off and on review, discussion/debate, nitpicking, commits, steps forward and backwards, to have someone swoop in at this time with a new raft of nitpicking and stylistic issues that set the review back again further is beyond maddening. And that's why you have not received much cooperation from me. Perhaps I should have stated that up front, but I've lost patience defending myself. Contributing to this PR is a tax on what is completely voluntary, unpaid time. I have no professional responsibility to this effort. Maybe it's better off done by someone who does.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    let me take a look this today. Thanks!


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/444/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91679/
    Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #93397 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93397/testReport)** for PR 21320 at commit [`9a1f808`](https://github.com/apache/spark/commit/9a1f80817abfa03430594ef3a9447f030f885bd5).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1204/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91678 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91678/testReport)** for PR 21320 at commit [`8ead76e`](https://github.com/apache/spark/commit/8ead76e3d37ac65a15268fafc61f9df2f90a5779).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class ContactWithDataPartitionColumn(name: FullName, address: String, pets: Int,`
      * `  case class BriefContactWithDataPartitionColumn(name: Name, address: String, p: Int)`


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94502 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94502/testReport)** for PR 21320 at commit [`3a833db`](https://github.com/apache/spark/commit/3a833db898d6068c8eda11a635d9053a5bb471d7).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r189491559
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala ---
    @@ -879,6 +879,15 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext
           }
         }
       }
    +
    +  test("select function over nested data") {
    --- End diff --
    
    Without this PR, this test still can pass, right? 
    
    Could you submit a separate PR for these test coverage improvement? We really welcome the test coverage improvement PRs.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by ajacques <gi...@git.apache.org>.
Github user ajacques commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205329769
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala ---
    @@ -0,0 +1,134 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * A Scala extractor that builds a [[org.apache.spark.sql.types.StructField]] from a Catalyst
    + * complex type extractor. For example, consider a relation with the following schema:
    + *
    + *   {{{
    + *   root
    + *    |-- name: struct (nullable = true)
    + *    |    |-- first: string (nullable = true)
    + *    |    |-- last: string (nullable = true)
    + *    }}}
    + *
    + * Further, suppose we take the select expression `name.first`. This will parse into an
    + * `Alias(child, "first")`. Ignoring the alias, `child` matches the following pattern:
    + *
    + *   {{{
    + *   GetStructFieldObject(
    + *     AttributeReference("name", StructType(_), _, _),
    + *     StructField("first", StringType, _, _))
    + *   }}}
    + *
    + * [[SelectedField]] converts that expression into
    + *
    + *   {{{
    + *   StructField("name", StructType(Array(StructField("first", StringType))))
    + *   }}}
    + *
    + * by mapping each complex type extractor to a [[org.apache.spark.sql.types.StructField]] with the
    + * same name as its child (or "parent" going right to left in the select expression) and a data
    + * type appropriate to the complex type extractor. In our example, the name of the child expression
    + * is "name" and its data type is a [[org.apache.spark.sql.types.StructType]] with a single string
    + * field named "first".
    + *
    + * @param expr the top-level complex type extractor
    + */
    +object SelectedField {
    +  def unapply(expr: Expression): Option[StructField] = {
    --- End diff --
    
    ```
    Error:(61, 12) constructor cannot be instantiated to expected type;
     found   : org.apache.spark.sql.catalyst.expressions.Alias
     required: org.apache.spark.sql.catalyst.expressions.ExtractValue
          case Alias(child, _) => child
    ```
    
    Alias takes: `Alias(child: Expression, name: String)`


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r190491050
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala ---
    @@ -879,6 +879,15 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext
           }
         }
       }
    +
    +  test("select function over nested data") {
    --- End diff --
    
    I did some forensics to understand the origin and reason for this test. It was part of the first commit of my original PR almost two years ago. Even then, this test passes without the rest of the commit. So I can't say why I added this test except perhaps that I felt it was useful code coverage.
    
    In any case, if you don't think it's a valuable contribution in this PR I'd rather just remove it entirely.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by jainaks <gi...@git.apache.org>.
Github user jainaks commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Hi @mallman ,
    I found another major issue after having this fix.
    Schema:
    a: struct (nullable = true)
     |    |-- b: struct (nullable = true)
     |    |    |-- c1: string (nullable = true)
     |    |    |-- c2: string (nullable = true)
     |    |    |-- c3: string (nullable = true)
     |    |    |-- c4: string (nullable = true)
     |    |    |-- c5: boolean (nullable = true)
    id: struct (nullable = true)
     |    |-- i1: struct (nullable = true)
     |    |    |-- i2: string (nullable = true)
    timestamp: bigint
    **Query:**
    select	a.b.c3 as c3, 
    		first(a.b.c3) over (partition by id.i1.i2 order by timestamp rows between current row and unbounded following) as first_c3
    from 	temp;
    The column "first_c3" gets the value of column "c2".
    It works well, if i just turn the parquetSchemaPrunning flag to false.
    It may sound odd in the first look and so does for me, but this is what i am getting.
    PS: I am running all my tests using #16578 pr.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r189493986
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,154 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, partitionSchema,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionFields = projects.flatMap(getFields)
    +        val filterFields = filters.flatMap(getFields)
    +        val requestedFields = (projectionFields ++ filterFields).distinct
    +
    +        // If [[requestedFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedFields.exists { case (_, optAtt) => optAtt.isEmpty }) {
    +          val prunedSchema = requestedFields
    +            .map { case (field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    +          val prunedDataSchema =
    +            StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name)))
    +
    +          // If the data schema is different from the pruned data schema, continue. Otherwise,
    +          // return [[op]]. We effect this comparison by counting the number of "leaf" fields in
    +          // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields
    +          // in [[dataSchema]].
    +          if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
    +            val prunedParquetRelation =
    +              hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession)
    +
    +            // We need to replace the expression ids of the pruned relation output attributes
    +            // with the expression ids of the original relation output attributes so that
    +            // references to the original relation's output are not broken
    +            val outputIdMap = l.output.map(att => (att.name, att.exprId)).toMap
    +            val prunedRelationOutput =
    +              prunedParquetRelation
    +                .schema
    +                .toAttributes
    +                .map {
    +                  case att if outputIdMap.contains(att.name) =>
    +                    att.withExprId(outputIdMap(att.name))
    +                  case att => att
    +                }
    +            val prunedRelation =
    +              l.copy(relation = prunedParquetRelation, output = prunedRelationOutput)
    +
    +            val projectionOverSchema = ProjectionOverSchema(prunedDataSchema)
    +
    +            // Construct a new target for our projection by rewriting and
    +            // including the original filters where available
    +            val projectionChild =
    +              if (filters.nonEmpty) {
    +                val projectedFilters = filters.map(_.transformDown {
    +                  case projectionOverSchema(expr) => expr
    +                })
    +                val newFilterCondition = projectedFilters.reduce(And)
    +                Filter(newFilterCondition, prunedRelation)
    +              } else {
    +                prunedRelation
    +              }
    +
    +            val nonDataPartitionColumnNames =
    +              partitionSchema.map(_.name).filterNot(dataSchemaFieldNames.contains).toSet
    --- End diff --
    
    Do we have any test case for covering this?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91441 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91441/testReport)** for PR 21320 at commit [`de06e81`](https://github.com/apache/spark/commit/de06e81495594a36e5c6c98111f7832a75c5ddc1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r204205744
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -71,9 +80,22 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
           StructType.fromString(schemaString)
         }
     
    -    val parquetRequestedSchema =
    +    val clippedParquetSchema =
           ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema)
     
    +    val parquetRequestedSchema = if (parquetMrCompatibility) {
    +      // Parquet-mr will throw an exception if we try to read a superset of the file's schema.
    --- End diff --
    
    What kind of exception does it throw? I thought Parquet supports it so far. Do you mean it doesn't support nested clipping schema between Parquet and Catalyst's? Which case does it now work?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92272/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94224/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @gatorsmile The last build was killed by SIGKILL. Can you start a new build, please?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @gatorsmile I've removed the changes to the files as you requested. This removes support for schema pruning on filters of queries. I've pushed the previous revision to a new branch in our `spark-public` repo, [spark-4502-parquet_column_pruning-filtering](https://github.com/VideoAmp/spark-public/tree/spark-4502-parquet_column_pruning-filtering).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94224 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94224/testReport)** for PR 21320 at commit [`a87b589`](https://github.com/apache/spark/commit/a87b589c70af69d25424b03b52e29165ab4e0b5f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman,
    
    > If I remove the changes to ParquetReadSupport.scala, then four tests fail in ParquetSchemaPruningSuite.scala.
    
    I think if the tests are few, you can make them ignored for now here, and make another PR enabling it back with the changes in `ParquetReadSupport.scala`.



---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91443/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by dbtsai <gi...@git.apache.org>.
Github user dbtsai commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    A [PR](https://github.com/apache/spark/pull/22357) fixing the issue I mentioned above  is provided by [Liang-Chi Hsieh](https://github.com/viirya). Thank you for the quick and clean solution.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #95185 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95185/testReport)** for PR 21320 at commit [`e6baf68`](https://github.com/apache/spark/commit/e6baf681e06e229d740af120491d1bf0f426af99).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91442 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91442/testReport)** for PR 21320 at commit [`a8e080f`](https://github.com/apache/spark/commit/a8e080fc8853e67367a60078d18758c0c4395c9b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > @mallman, can we close this PR? Are you willing to update here or not?
    
    I pushed an update less than a day ago, and I intend to continue pushing updates as needed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Hi @Gauravshah. That branch has diverged substantially from what’s in master. Right now I’m preparing a  PR to address a problem with the current implementation in master, but I’m on holiday for a while.
    
    Still, I am hopeful we will see schema pruning for joins and aggregations in 3.0.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3920/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/90582/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91441/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Sorry it's taken me a couple of days to respond. I needed the time to ruminate (and not). I could write voluminously, but I just want to reply to a couple of points and move on.
    
    > I think that's primarily because the change looks incomplete but the feature itself sounds good to have. I think that's why people try to take a look a lot.
    
    It's very incomplete, yes, in comparison to where it started in #16578. That PR supported pruning in projections, filters, joins and aggregations, and paved the way for further optimizations—e.g. window support. This specific PR started with projections and filters, then removed support for filters, then removed code that ensured it wouldn't fail under certain scenarios (r.e. the now ignored tests). I will grant that the latter code definitely falls under the "gee this seems to fix it" kind of workaround than an actually correct fix, and I'm not against pursuing the latter. I am uneasy about merging in broken code.
    
    I have complied with @gatorsmile's requests to remove changes from this PR. Sometimes in the process I have accidentally left some dangling dead code or what appears to be some bad design decisions, such as in https://github.com/apache/spark/pull/21320#issuecomment-407713622. In all of those cases, this is code that makes more sense in the broader context of #16578. I could have just kept quiet and complied, or defended myself, but I didn't have patience for the former nor energy for the latter. At this point, after about a week of time I am in a better mood to collaborate.
    
    > Also honestly this logic looks convoluted.
    
    I need to point out that I don't know how to respond to comments like these. I have put my best effort forward. If you want me to change something, I need more specific guidance. What do you want me to do about this? Don't just point out a problem, offer a solution or at least a suggestion.
    
    > @ajacques, if you are willing to take over this, please go ahead. I would appreciate it.
    
    This is probably the best way to get through this PR. I will try to participate and help. Please post a link to your PR when you open it.
    
    Thanks.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @gatorsmile Any concerns about merging this PR at this point?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/33/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91441 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91441/testReport)** for PR 21320 at commit [`de06e81`](https://github.com/apache/spark/commit/de06e81495594a36e5c6c98111f7832a75c5ddc1).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #95041 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95041/testReport)** for PR 21320 at commit [`2711746`](https://github.com/apache/spark/commit/271174637f35aa5684ec6cc1938c4c8b210553d3).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by jainaks <gi...@git.apache.org>.
Github user jainaks commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Thanks @mallman for making this huge contribution. 3 years is really a long time to keep patience for concluding things.
    I am attaching the sample parquet file for your reference with which you can reproduce the Window function, wrong column selection issue.
    
    [sample.parquet.txt](https://github.com/apache/spark/files/2230873/sample.parquet.txt)
    Please remove .txt from the filename.
    
    Following are the simple steps you can follow to reproduce this issue via spark shell.
    ```
    import org.apache.spark.sql.SparkSession
    val ss = SparkSession.builder().config("spark.sql.nestedSchemaPruning.enabled", "true").getOrCreate()
    val inputdf = ss.read.parquet("sample.parquet")
    inputdf.createOrReplaceTempView(“temptable”)
    ss.sql("select page.url, first(page.url) over (partition by id order by timestamp rows between current row and unbounded following) from temptable").collect.foreach(println)
    ```
    Result:
    `[https://adobeid-na1.services.adobe.com/renga-idprovider/pages/login,Account:IMS:onLoad_SignInForm]`
    Please let me know if you need any help from my side.
    
    PS: Sorry for responding late to it. Few high priority items kept me busy.



---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Sure, @gatorsmile !


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199354841
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -417,11 +417,12 @@ class ParquetFileFormat
           } else {
             logDebug(s"Falling back to parquet-mr")
             // ParquetRecordReader returns UnsafeRow
    +        val readSupport = new ParquetReadSupport(convertTz, true)
    --- End diff --
    
    > val readSupport = new ParquetReadSupport(convertTz, parquetMrCompatibility = true)


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @gatorsmile The last couple of build test failures appear to be entirely unrelated to this PR. The error message in the one failed test reads `org.scalatest.exceptions.TestFailedException: Unable to download Spark 2.2.0`. Please run a new test build when you get back to the review if this is still the last problem with this PR's test build.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205022895
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionRootFields = projects.flatMap(getRootFields)
    +        val filterRootFields = filters.flatMap(getRootFields)
    +        val requestedRootFields = (projectionRootFields ++ filterRootFields).distinct
    +
    +        // If [[requestedRootFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) {
    +          val prunedSchema = requestedRootFields
    +            .map { case RootField(field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    +          val prunedDataSchema =
    +            StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name)))
    +
    +          // If the data schema is different from the pruned data schema, continue. Otherwise,
    +          // return [[op]]. We effect this comparison by counting the number of "leaf" fields in
    +          // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields
    +          // in [[dataSchema]].
    +          if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
    --- End diff --
    
    No comment.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r190493689
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,154 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, partitionSchema,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionFields = projects.flatMap(getFields)
    +        val filterFields = filters.flatMap(getFields)
    +        val requestedFields = (projectionFields ++ filterFields).distinct
    +
    +        // If [[requestedFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedFields.exists { case (_, optAtt) => optAtt.isEmpty }) {
    +          val prunedSchema = requestedFields
    +            .map { case (field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    +          val prunedDataSchema =
    +            StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name)))
    +
    +          // If the data schema is different from the pruned data schema, continue. Otherwise,
    +          // return [[op]]. We effect this comparison by counting the number of "leaf" fields in
    +          // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields
    +          // in [[dataSchema]].
    +          if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
    +            val prunedParquetRelation =
    +              hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession)
    +
    +            // We need to replace the expression ids of the pruned relation output attributes
    +            // with the expression ids of the original relation output attributes so that
    +            // references to the original relation's output are not broken
    +            val outputIdMap = l.output.map(att => (att.name, att.exprId)).toMap
    +            val prunedRelationOutput =
    +              prunedParquetRelation
    +                .schema
    +                .toAttributes
    +                .map {
    +                  case att if outputIdMap.contains(att.name) =>
    +                    att.withExprId(outputIdMap(att.name))
    +                  case att => att
    +                }
    +            val prunedRelation =
    +              l.copy(relation = prunedParquetRelation, output = prunedRelationOutput)
    +
    +            val projectionOverSchema = ProjectionOverSchema(prunedDataSchema)
    +
    +            // Construct a new target for our projection by rewriting and
    +            // including the original filters where available
    +            val projectionChild =
    +              if (filters.nonEmpty) {
    +                val projectedFilters = filters.map(_.transformDown {
    +                  case projectionOverSchema(expr) => expr
    +                })
    +                val newFilterCondition = projectedFilters.reduce(And)
    +                Filter(newFilterCondition, prunedRelation)
    +              } else {
    +                prunedRelation
    +              }
    +
    +            val nonDataPartitionColumnNames =
    +              partitionSchema.map(_.name).filterNot(dataSchemaFieldNames.contains).toSet
    --- End diff --
    
    You mean a test case in which the table under test has partition columns as part of its data schema? I don't think so, however I will check and work on adding any missing test coverage. LMK if this is not what you're asking about.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r204289193
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionRootFields = projects.flatMap(getRootFields)
    +        val filterRootFields = filters.flatMap(getRootFields)
    +        val requestedRootFields = (projectionRootFields ++ filterRootFields).distinct
    +
    +        // If [[requestedRootFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) {
    +          val prunedSchema = requestedRootFields
    +            .map { case RootField(field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    +          val prunedDataSchema =
    +            StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name)))
    +
    +          // If the data schema is different from the pruned data schema, continue. Otherwise,
    +          // return [[op]]. We effect this comparison by counting the number of "leaf" fields in
    +          // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields
    +          // in [[dataSchema]].
    +          if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
    --- End diff --
    
    Can we make some private functions for these? Looks hard to follow.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Seems fine to me too for the similar reasons.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Add some test cases when turning on spark.sql.caseSensitive?
    
    Will do.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205021282
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * A Scala extractor that projects an expression over a given schema. Data types,
    + * field indexes and field counts of complex type extractors and attributes
    + * are adjusted to fit the schema. All other expressions are left as-is. This
    + * class is motivated by columnar nested schema pruning.
    + */
    +case class ProjectionOverSchema(schema: StructType) {
    --- End diff --
    
    Okay. So...


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by Gauravshah <gi...@git.apache.org>.
Github user Gauravshah commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    backported to 2.3.2 just in case somebody needs it. https://github.com/Gauravshah/spark/tree/branch-2.3_SPARK-4502 Thanks @mallman 


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94969 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94969/testReport)** for PR 21320 at commit [`3056896`](https://github.com/apache/spark/commit/3056896fbd38bd8733451d63fbe2336aac173651).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #93390 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93390/testReport)** for PR 21320 at commit [`d027eef`](https://github.com/apache/spark/commit/d027eefab2b0d84f86f2a61648e646e0c8a8d3d0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    I said either way works fine. It doesn't matter which way we go. Better close one of them if the approach is the same and both PRs are active.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Thanks everyone for your contributions, support and patience. It's been a journey and a half, and I'm excited for the future. I will open a follow-on PR to address the current known failure scenario (see ignored test) in this patch, and we can discuss if/how we can get it into 2.4 as well.
    
    I know there are many early adopters of this patch and #16578. Bug reports will continue to be very helpful.
    
    Beyond this patch, there are many possibilities for widening the scope of schema pruning. As part of our review process, we've pared the scope of this capability to just projection. IMHO, the first limitation we should address post 2.4 is supporting pruning with query filters of nested fields ("where" clauses). Joins, aggregations and window queries would be powerful enhancements as well, bringing the scope of schema pruning to analytic queries.
    
    I believe all of the additional features VideoAmp has implemented for schema pruning are independent of the underlying column store. Future enhancements should be automagically inherited by any column store that implements functionality analogous to `ParquetSchemaPruning.scala`. This should widen not just the audience that can be reached, but the developer community that can contribute and review.
    
    Thanks again.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > @gatorsmile @HyukjinKwon @ajacques I'm seeing incorrect results from the following simple projection query, given @jainaks test file:
    >
    >```
    >select page.url, page from temptable
    >```
    
    I believe I have identified and fixed this bug. I have added a commit with a fix and additional test cases that cover it and related failure scenarios.
    
    The underlying problem was that when merging the struct fields derived from the projections `page.url` and `page` into a single pruned schema, the merge function we are using does not necessarily respect the order of the fields in the schema of `temptable`. In the above and similar scenarios, the merge function merged `page.url` and `page` into a struct consisting of the `page.url` field followed by the other fields in `page`. While this produces a schema that has a subset of the fields of `temptable`'s schema, the fields are in the wrong order.
    
    I considered two high-level approaches to fixing this problem. The first was to rework the way the pruned schema was constructed in the first place. That is, rather than construct the pruned schema by merging fields together, construct the merged schema by directly filtering the original schema. I think that approach would go along the lines of altering the `SelectedField` extractor for an expression to a `SelectedStruct` extractor that extracts a whole struct from a sequence of expressions. The latter expressions would consist of the projection and filtering expressions of a `PhysicalOperation`. I did not go further in exploring that route, as it would involve a substantial rewrite of the patch. However, in the end it may be the cleaner, more natural route.
    
    The second approach I considered—and adopted—was to sort the fields of the merged schema, recursively, so that the order of the fields in the merged schema respected the order of their namesakes in the original schema. This adds more complexity to the patch, undesirable for something already so complex. But it appeared to be the quickest route to a correct solution.
    
    Given more time, I would probably explore rewriting the patch with a `SelectedStruct` extractor as described above. I don't know it would actually lead to something less complex. It's just a thought.
    
    I added three additional test "scenarios" to `ParquetSchemaPruningSuite.scala` (each "scenario" is tested four ways by the `testSchemaPruning` function). They test three distinct scenarios that fail without the fix. These scenarios consist of a field and its parent struct, an array-based variation and a map-based variation. I added some additional array and map data to the test data to ensure proper test coverage.
    
    Incidentally, I also added an integer `id` field to the test contact types so that the results of queries on the contact tables can be ordered deterministically. This should have been part of the tests all along, but I forgot to incorporate it.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93384/
    Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Hi @jainaks. Thanks for your report. Do you have the same problem running your test with this PR?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91678 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91678/testReport)** for PR 21320 at commit [`8ead76e`](https://github.com/apache/spark/commit/8ead76e3d37ac65a15268fafc61f9df2f90a5779).


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r204607379
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala ---
    @@ -0,0 +1,388 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.scalatest.BeforeAndAfterAll
    +import org.scalatest.exceptions.TestFailedException
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.dsl.plans._
    +import org.apache.spark.sql.catalyst.expressions.NamedExpression
    +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
    +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
    +import org.apache.spark.sql.types._
    +
    +// scalastyle:off line.size.limit
    +class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll {
    +  // The test schema as a tree string, i.e. `schema.treeString`
    +  // root
    +  //  |-- col1: string (nullable = false)
    +  //  |-- col2: struct (nullable = true)
    +  //  |    |-- field1: integer (nullable = true)
    +  //  |    |-- field2: array (nullable = true)
    +  //  |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field3: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield3: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = true)
    +  //  |    |-- field4: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field5: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: struct (nullable = false)
    +  //  |    |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: struct (nullable = true)
    +  //  |    |    |    |    |-- subsubfield1: struct (nullable = true)
    +  //  |    |    |    |    |    |-- subsubsubfield1: string (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field6: struct (nullable = true)
    +  //  |    |    |-- subfield1: string (nullable = false)
    +  //  |    |    |-- subfield2: string (nullable = true)
    +  //  |    |-- field7: struct (nullable = true)
    +  //  |    |    |-- subfield1: struct (nullable = true)
    +  //  |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field8: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: array (valueContainsNull = false)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field9: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col3: array (nullable = false)
    +  //  |    |-- element: struct (containsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col4: map (nullable = false)
    +  //  |    |-- key: string
    +  //  |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col5: array (nullable = true)
    +  //  |    |-- element: map (containsNull = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col6: map (nullable = true)
    +  //  |    |-- key: string
    +  //  |    |-- value: array (valueContainsNull = true)
    +  //  |    |    |-- element: struct (containsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col7: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: integer (nullable = false)
    +  //  |    |    |-- field2: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |-- field3: array (nullable = true)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |-- col8: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: array (nullable = false)
    +  //  |    |    |    |-- element: integer (containsNull = false)
    +  private val schema =
    +    StructType(
    +      StructField("col1", StringType, nullable = false) ::
    +      StructField("col2", StructType(
    +        StructField("field1", IntegerType) ::
    +        StructField("field2", ArrayType(IntegerType, containsNull = false)) ::
    +        StructField("field3", ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", IntegerType) ::
    +          StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) ::
    +        StructField("field4", MapType(StringType, StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) ::
    +        StructField("field5", ArrayType(StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil), nullable = false) ::
    +          StructField("subfield2", StructType(
    +            StructField("subsubfield1", StructType(
    +              StructField("subsubsubfield1", StringType) :: Nil)) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)), nullable = false) ::
    +        StructField("field6", StructType(
    +          StructField("subfield1", StringType, nullable = false) ::
    +          StructField("subfield2", StringType) :: Nil)) ::
    +        StructField("field7", StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) ::
    +        StructField("field8", MapType(StringType, ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil)), valueContainsNull = false)) ::
    +        StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) ::
    +     StructField("col3", ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) ::
    +     StructField("col4", MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) ::
    +     StructField("col5", ArrayType(MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) ::
    +     StructField("col6", MapType(StringType, ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) ::
    +     StructField("col7", ArrayType(StructType(
    +       StructField("field1", IntegerType, nullable = false) ::
    +       StructField("field2", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil)) ::
    +       StructField("field3", ArrayType(StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) ::
    +     StructField("col8", ArrayType(StructType(
    +       StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) :: Nil)
    +
    +  private val testRelation = LocalRelation(schema.toAttributes)
    --- End diff --
    
    Per https://github.com/apache/spark/pull/21320#discussion_r203933423
    
    Can we make multiple relations with separate smaller schemas to make it more readable?


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r197629698
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala ---
    @@ -301,7 +301,6 @@ case class FileSourceScanExec(
         } getOrElse {
           withOptPartitionCount
         }
    -
    --- End diff --
    
    Remove this line?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman and @ajacques, if you guys find it's any difficulty, I will take over this. Please review this. Let me know if you guys think that's better way to get through this.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > @mallman It does work fine with "name.First".
    
    @jainaks What is the value of the Spark SQL configuration setting `spark.sql.caseSensitive` when you run this query? Also, are you querying the parquet file as part of a Hive metastore table or from a dataframe loaded with the [DataFrameReader.parquet](http://spark.apache.org/docs/latest/api/scala/index.html#org.apache.spark.sql.DataFrameReader@parquet(path:String):org.apache.spark.sql.DataFrame) method?


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r203934281
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -47,16 +47,25 @@ import org.apache.spark.sql.types._
      *
      * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]]
      * to [[prepareForRead()]], but use a private `var` for simplicity.
    + *
    + * @param parquetMrCompatibility support reading with parquet-mr or Spark's built-in Parquet reader
      */
    -private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
    +private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone],
    +    parquetMrCompatibility: Boolean)
         extends ReadSupport[UnsafeRow] with Logging {
       private var catalystRequestedSchema: StructType = _
     
    +  /**
    +   * Construct a [[ParquetReadSupport]] with [[convertTz]] set to [[None]] and
    +   * [[parquetMrCompatibility]] set to [[false]].
    +   *
    +   * We need a zero-arg constructor for SpecificParquetRecordReaderBase.  But that is only
    +   * used in the vectorized reader, where we get the convertTz value directly, and the value here
    +   * is ignored. Further, we set [[parquetMrCompatibility]] to [[false]] as this constructor is only
    +   * called by the Spark reader.
    --- End diff --
    
    re: https://github.com/apache/spark/pull/21320/files/cb858f202e49d69f2044681e37f982dc10676296#r199631341 actually, it doesn't looks clear to me too. What does the flag indicate? you mean normal parquet reader vs vectorized parquet reader?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91679 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91679/testReport)** for PR 21320 at commit [`89febc8`](https://github.com/apache/spark/commit/89febc8e978d606e32911088e9589462805b8697).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @gatorsmile I've addressed many of your points in today's commits. Can you please take a look at what I've done so far? I'm still working on the PRs you requested.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Then should we keep this one or https://github.com/apache/spark/pull/21889? shall we deduplicate the efforts? I requested to open that because this looks going to be inactive per your comments.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    >> the window of opportunity to review syntax and style in this PR closed long ago.
    > Why/when is this window closed? Who closed that?
    
    What I wrote above is a coarse approximation of my stance on the matter. It's inadequate, and I have struggled to adequately express myself. Reflecting on this last night I believe I was able to nail down exactly what I want to write, but I don't have time to write right now. I will reply in full later, within a day or two. I will also address your recent comments and questions.
    
    Thank you.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Try this when spark.sql.nestedSchemaPruning.enabled is on?
    
    I don't think this will be difficult to fix. I'm working on it now and will add relevant test coverage.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r203931755
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * A Scala extractor that projects an expression over a given schema. Data types,
    + * field indexes and field counts of complex type extractors and attributes
    + * are adjusted to fit the schema. All other expressions are left as-is. This
    + * class is motivated by columnar nested schema pruning.
    + */
    +case class ProjectionOverSchema(schema: StructType) {
    --- End diff --
    
    This still looks weird that we place this under `catalyst` since we currently only use it under `execution`.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r190492424
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,154 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, partitionSchema,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionFields = projects.flatMap(getFields)
    +        val filterFields = filters.flatMap(getFields)
    +        val requestedFields = (projectionFields ++ filterFields).distinct
    +
    +        // If [[requestedFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedFields.exists { case (_, optAtt) => optAtt.isEmpty }) {
    +          val prunedSchema = requestedFields
    +            .map { case (field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    +          val prunedDataSchema =
    +            StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name)))
    +
    +          // If the data schema is different from the pruned data schema, continue. Otherwise,
    +          // return [[op]]. We effect this comparison by counting the number of "leaf" fields in
    +          // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields
    +          // in [[dataSchema]].
    +          if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
    +            val prunedParquetRelation =
    +              hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession)
    +
    +            // We need to replace the expression ids of the pruned relation output attributes
    +            // with the expression ids of the original relation output attributes so that
    +            // references to the original relation's output are not broken
    +            val outputIdMap = l.output.map(att => (att.name, att.exprId)).toMap
    +            val prunedRelationOutput =
    +              prunedParquetRelation
    +                .schema
    +                .toAttributes
    +                .map {
    +                  case att if outputIdMap.contains(att.name) =>
    +                    att.withExprId(outputIdMap(att.name))
    +                  case att => att
    +                }
    +            val prunedRelation =
    +              l.copy(relation = prunedParquetRelation, output = prunedRelationOutput)
    +
    +            val projectionOverSchema = ProjectionOverSchema(prunedDataSchema)
    +
    +            // Construct a new target for our projection by rewriting and
    +            // including the original filters where available
    +            val projectionChild =
    +              if (filters.nonEmpty) {
    +                val projectedFilters = filters.map(_.transformDown {
    +                  case projectionOverSchema(expr) => expr
    +                })
    +                val newFilterCondition = projectedFilters.reduce(And)
    +                Filter(newFilterCondition, prunedRelation)
    +              } else {
    +                prunedRelation
    +              }
    +
    +            val nonDataPartitionColumnNames =
    +              partitionSchema.map(_.name).filterNot(dataSchemaFieldNames.contains).toSet
    +
    +            // Construct the new projections of our [[Project]] by
    +            // rewriting the original projections
    +            val newProjects = projects.map {
    +              case project if (nonDataPartitionColumnNames.contains(project.name)) => project
    +              case project =>
    +                (project transformDown {
    +                  case projectionOverSchema(expr) => expr
    +                }).asInstanceOf[NamedExpression]
    +            }
    +
    +            logDebug("New projects:\n" + newProjects.map(_.treeString).mkString("\n"))
    +            logDebug(s"Pruned data schema:\n${prunedDataSchema.treeString}")
    +
    +            Project(newProjects, projectionChild)
    +          } else {
    +            op
    +          }
    +        } else {
    +          op
    +        }
    +    }
    +
    +  /**
    +   * Gets the top-level (no-parent) [[StructField]]s for the given [[Expression]].
    +   * When [[expr]] is an [[Attribute]], construct a field around it and return the
    +   * attribute as the second component of the returned tuple.
    +   */
    +  private def getFields(expr: Expression): Seq[(StructField, Option[Attribute])] = {
    --- End diff --
    
    Will do.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by ajacques <gi...@git.apache.org>.
Github user ajacques commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Hey @mallman, I want to thank you for your work on this so far. I've been watching this pull request hoping this would get merged into 2.4 since it would be a benefit to me, but can see how it might be frustrating.
    
    Unfortunately, I've only been following the comments and not the code/architecture itself, so I can't take over effectively, but I did try to make the minor comments as requested hopefully to help out. I've started in 7ee616076f93d6cfd55b6646314f3d4a6d1530d3. This may not be super helpful right now, but if these were the only blockers for getting this change into mainline in time for 2.4.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    cc @viirya since I noticed you're not cc'ed here.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r189479383
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -1256,8 +1256,18 @@ object SQLConf {
             "issues. Turn on this config to insert a local sort before actually doing repartition " +
             "to generate consistent repartition results. The performance of repartition() may go " +
             "down since we insert extra local sort before it.")
    +        .booleanConf
    +        .createWithDefault(true)
    +
    +  val NESTED_SCHEMA_PRUNING_ENABLED =
    +    buildConf("spark.sql.nestedSchemaPruning.enabled")
    +      .internal()
    +      .doc("Prune nested fields from a logical relation's output which are unnecessary in " +
    +        "satisfying a query. This optimization allows columnar file format readers to avoid " +
    +        "reading unnecessary nested column data. Currently Parquet is the only data source that " +
    --- End diff --
    
    How about ORC? 
    
    cc @dongjoon-hyun Do you know whether it is also doable in the latest ORC version?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman, can we close this PR? Are you willing to update here or not?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94476 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94476/testReport)** for PR 21320 at commit [`3a833db`](https://github.com/apache/spark/commit/3a833db898d6068c8eda11a635d9053a5bb471d7).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    >> Hi @jainaks. Thanks for your report. Do you have the same problem running your test with this PR?
    > @mallman Yes, the issue with window functions is reproducible even with this PR.
    
    Hi @jainaks. I want to circle back with you about this. Are you still having this trouble with the latest version of the PR? If so, can you please attach a small parquet file for testing?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > I think if the tests are few, you can make them ignored for now here, and make another PR enabling it back with the changes in ParquetReadSupport.scala.
    
    That's the approach I've taken in the last rebase I just pushed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205063684
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * A Scala extractor that projects an expression over a given schema. Data types,
    + * field indexes and field counts of complex type extractors and attributes
    + * are adjusted to fit the schema. All other expressions are left as-is. This
    + * class is motivated by columnar nested schema pruning.
    + */
    +case class ProjectionOverSchema(schema: StructType) {
    --- End diff --
    
    can we move it to `catalyst`?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94235/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199365004
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -47,16 +47,25 @@ import org.apache.spark.sql.types._
      *
      * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]]
      * to [[prepareForRead()]], but use a private `var` for simplicity.
    + *
    + * @param parquetMrCompatibility support reading with parquet-mr or Spark's built-in Parquet reader
    --- End diff --
    
    The description is not clear. Could you make it better?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92191/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r212476268
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ---
    @@ -31,6 +32,7 @@ import org.apache.spark.sql.types.StructType
     class ParquetSchemaPruningSuite
         extends QueryTest
         with ParquetTest
    +    with SchemaPruningTest
    --- End diff --
    
    Why this is removed?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Yup, for example,
    
    https://github.com/apache/spark/pull/21320#discussion_r203931755
    https://github.com/apache/spark/pull/21320#discussion_r203932334
    https://github.com/apache/spark/pull/21320#discussion_r203933307
    https://github.com/apache/spark/pull/21320#discussion_r203933423
    https://github.com/apache/spark/pull/21320#discussion_r203934281
    https://github.com/apache/spark/pull/21320#issuecomment-407713622


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r212194825
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala ---
    @@ -202,11 +204,15 @@ private[parquet] class ParquetRowConverter(
     
       override def start(): Unit = {
         var i = 0
    -    while (i < currentRow.numFields) {
    +    while (i < fieldConverters.length) {
           fieldConverters(i).updater.start()
           currentRow.setNullAt(i)
           i += 1
         }
    +    while (i < currentRow.numFields) {
    --- End diff --
    
    @mallman It sounds like the changes in this file are not needed. Could you help me point out which test cases will fail?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #4277 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4277/testReport)** for PR 21320 at commit [`0e5594b`](https://github.com/apache/spark/commit/0e5594b6ac1dcb94f3f0166e66a7d4e7eae3d00c).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1200/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by jainaks <gi...@git.apache.org>.
Github user jainaks commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Hi @mallman, Thanks for this PR. It has huge impact on performance, when querying the nested parquet schema. I had used the original PR#16578 and found an issue, that it does not works well when the query has column names in different case.
    e.g. the schema is:
    root
     |-- name: struct
     |    |-- First: string
     |    |-- Last: string
     |-- address: string
    and if i put a join query, referring the column as "NAME.first".
    It throws an exception:
    **ERROR:  org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: NAME#137322**
    If you want, i can share the exact schema and query for debugging.
    Though, i have fixed this in my local repo and get it working fine. 
    I have commented on the exact code line, which causes this issue. 
    Please let me know if you want me to share the fix.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r203933307
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ---
    @@ -0,0 +1,156 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import java.io.File
    +
    +import org.apache.spark.sql.{QueryTest, Row}
    +import org.apache.spark.sql.execution.FileSchemaPruningTest
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class ParquetSchemaPruningSuite
    +    extends QueryTest
    +    with ParquetTest
    +    with FileSchemaPruningTest
    +    with SharedSQLContext {
    --- End diff --
    
    Can we just simply:
    
    ```scala
      override def beforeEach(): Unit = {
        super.beforeEach()
        spark.conf.set(SQLConf. NESTED_SCHEMA_PRUNING_ENABLED.key, "true")
      }
    
      override def afterEach(): Unit = {
        try {
          spark.conf.unset(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key)
        } finally {
          super.afterEach()
        }
      }
    ```
    
    without the complicated hierarchy?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman Yes, the issue with window functions is reproducible even with this PR.
    
    Can you attach a (small) parquet file I can use to test this scenario?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @ajacques please open a PR and let's get this in. Let me make a followup separately then.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91443 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91443/testReport)** for PR 21320 at commit [`0351094`](https://github.com/apache/spark/commit/0351094317aa9093fd4039320f1e6e3bba4c0dc5).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #95185 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95185/testReport)** for PR 21320 at commit [`e6baf68`](https://github.com/apache/spark/commit/e6baf681e06e229d740af120491d1bf0f426af99).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r189489061
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala ---
    @@ -162,7 +162,9 @@ case class FilterExec(condition: Expression, child: SparkPlan)
         val generatedIsNotNullChecks = new Array[Boolean](notNullPreds.length)
         val generated = otherPreds.map { c =>
           val nullChecks = c.references.map { r =>
    -        val idx = notNullPreds.indexWhere { n => n.asInstanceOf[IsNotNull].child.semanticEquals(r)}
    +        val idx = notNullPreds.indexWhere { n =>
    +          n.asInstanceOf[IsNotNull].child.references.contains(r)
    --- End diff --
    
    Is this change related?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94971/
    Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95185/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @HyukjinKwon We plan to merge this highly desirable feature into Spark 2.4 release. 


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by ajacques <gi...@git.apache.org>.
Github user ajacques commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @HyukjinKwon, I'm not totally familiar with Spark internals yet, so to be honest I don't feel confident making big changes and hopefully can keep them simple at first.
    
    I've gone through the code review comments and made as many changes as possible [here](https://github.com/apache/spark/compare/master...ajacques:spark-4502-parquet_column_pruning-foundation). If this PR is mostly feature complete and it's just small things, then I can push forward.
    
    If the feedback comments push past simple refactoring level right now I would prefer to let someone else take over, but feel free to use what I've done.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r212414888
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala ---
    @@ -202,11 +204,15 @@ private[parquet] class ParquetRowConverter(
     
       override def start(): Unit = {
         var i = 0
    -    while (i < currentRow.numFields) {
    +    while (i < fieldConverters.length) {
           fieldConverters(i).updater.start()
           currentRow.setNullAt(i)
           i += 1
         }
    +    while (i < currentRow.numFields) {
    --- End diff --
    
    My suggestion is to remove the changes from this file at first. We can handle the ignored cases in the follow-up PRs. 


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2291/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > BTW, I am trying to take a look closely. I would appreciate if there are some concrete examples so that I (or other reviewers) can double check along. Parquet is pretty core fix and let's be very sure on each case.
    
    What kind of examples would help?


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r190494243
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.scalatest.BeforeAndAfterAll
    +import org.scalatest.exceptions.TestFailedException
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.dsl.plans._
    +import org.apache.spark.sql.catalyst.expressions.NamedExpression
    +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
    +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
    +import org.apache.spark.sql.types._
    +
    +// scalastyle:off line.size.limit
    +class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll {
    +  // The test schema as a tree string, i.e. `schema.treeString`
    +  // root
    +  //  |-- col1: string (nullable = false)
    +  //  |-- col2: struct (nullable = true)
    +  //  |    |-- field1: integer (nullable = true)
    +  //  |    |-- field2: array (nullable = true)
    +  //  |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field3: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield3: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = true)
    +  //  |    |-- field4: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field5: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: struct (nullable = false)
    +  //  |    |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: struct (nullable = true)
    +  //  |    |    |    |    |-- subsubfield1: struct (nullable = true)
    +  //  |    |    |    |    |    |-- subsubsubfield1: string (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field6: struct (nullable = true)
    +  //  |    |    |-- subfield1: string (nullable = false)
    +  //  |    |    |-- subfield2: string (nullable = true)
    +  //  |    |-- field7: struct (nullable = true)
    +  //  |    |    |-- subfield1: struct (nullable = true)
    +  //  |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field8: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: array (valueContainsNull = false)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field9: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col3: array (nullable = false)
    +  //  |    |-- element: struct (containsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col4: map (nullable = false)
    +  //  |    |-- key: string
    +  //  |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col5: array (nullable = true)
    +  //  |    |-- element: map (containsNull = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col6: map (nullable = true)
    +  //  |    |-- key: string
    +  //  |    |-- value: array (valueContainsNull = true)
    +  //  |    |    |-- element: struct (containsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col7: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: integer (nullable = false)
    +  //  |    |    |-- field2: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |-- field3: array (nullable = true)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |-- col8: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: array (nullable = false)
    +  //  |    |    |    |-- element: integer (containsNull = false)
    +  private val schema =
    +    StructType(
    +      StructField("col1", StringType, nullable = false) ::
    +      StructField("col2", StructType(
    +        StructField("field1", IntegerType) ::
    +        StructField("field2", ArrayType(IntegerType, containsNull = false)) ::
    +        StructField("field3", ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", IntegerType) ::
    +          StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) ::
    +        StructField("field4", MapType(StringType, StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) ::
    +        StructField("field5", ArrayType(StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil), nullable = false) ::
    +          StructField("subfield2", StructType(
    +            StructField("subsubfield1", StructType(
    +              StructField("subsubsubfield1", StringType) :: Nil)) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)), nullable = false) ::
    +        StructField("field6", StructType(
    +          StructField("subfield1", StringType, nullable = false) ::
    +          StructField("subfield2", StringType) :: Nil)) ::
    +        StructField("field7", StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) ::
    +        StructField("field8", MapType(StringType, ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil)), valueContainsNull = false)) ::
    +        StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) ::
    +     StructField("col3", ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) ::
    +     StructField("col4", MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) ::
    +     StructField("col5", ArrayType(MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) ::
    +     StructField("col6", MapType(StringType, ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) ::
    +     StructField("col7", ArrayType(StructType(
    +       StructField("field1", IntegerType, nullable = false) ::
    +       StructField("field2", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil)) ::
    +       StructField("field3", ArrayType(StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) ::
    +     StructField("col8", ArrayType(StructType(
    +       StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) :: Nil)
    +
    +  private val testRelation = LocalRelation(schema.toAttributes)
    +
    +  test("should not match an attribute reference") {
    +    assertResult(None)(unapplySelect("col1"))
    +    assertResult(None)(unapplySelect("col1 as foo"))
    +    assertResult(None)(unapplySelect("col2"))
    +  }
    +
    +  test("col2.field2, col2.field2[0] as foo") {
    --- End diff --
    
    I'll work on making these test names more readable.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/392/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199389588
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala ---
    @@ -182,18 +182,20 @@ private[parquet] class ParquetRowConverter(
     
       // Converters for each field.
       private val fieldConverters: Array[Converter with HasParentContainerUpdater] = {
    -    parquetType.getFields.asScala.zip(catalystType).zipWithIndex.map {
    -      case ((parquetFieldType, catalystField), ordinal) =>
    -        // Converted field value should be set to the `ordinal`-th cell of `currentRow`
    -        newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal))
    +    parquetType.getFields.asScala.map {
    +      case parquetField =>
    +        val fieldIndex = catalystType.fieldIndex(parquetField.getName)
    --- End diff --
    
    The name can be used as the identifiers? Could you double check whether we can save the a parquet file with duplicate column names? [Note: the previous version of Spark does not check name duplication. Thus, I guess the previous version of Spark might generate the file with duplicate column names]


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    retest this please


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94502 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94502/testReport)** for PR 21320 at commit [`3a833db`](https://github.com/apache/spark/commit/3a833db898d6068c8eda11a635d9053a5bb471d7).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > https://issues.apache.org/jira/browse/SPARK-25879
    > 
    > If we select a nested field and a top level field, the schema pruning will fail. Here is the reproducible test,
    > ...
    
    Hi @dbtsai.
    
    I believe the problem you're seeing here is resolved by #22880 (https://issues.apache.org/jira/browse/SPARK-25407). It was a known problem at the time this PR was merged, but was pushed back to a future commit. Coincidentally, I just posted #22880 today.
    
    The test case you provide is very similar to the test case introduced and exercised in that PR. I manually ran your test case on that branch locally, and the test passed. Would you like to try that branch and comment?
    
    Cheers.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r201863353
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionRootFields = projects.flatMap(getRootFields)
    +        val filterRootFields = filters.flatMap(getRootFields)
    +        val requestedRootFields = (projectionRootFields ++ filterRootFields).distinct
    +
    +        // If [[requestedRootFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) {
    +          val prunedSchema = requestedRootFields
    +            .map { case RootField(field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    --- End diff --
    
    Yes.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199643803
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -71,9 +80,22 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
           StructType.fromString(schemaString)
         }
     
    -    val parquetRequestedSchema =
    +    val clippedParquetSchema =
           ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema)
     
    +    val parquetRequestedSchema = if (parquetMrCompatibility) {
    +      // Parquet-mr will throw an exception if we try to read a superset of the file's schema.
    +      // Therefore, we intersect our clipped schema with the underlying file's schema
    +      ParquetReadSupport.intersectParquetGroups(clippedParquetSchema, context.getFileSchema)
    +        .map(intersectionGroup =>
    +          new MessageType(intersectionGroup.getName, intersectionGroup.getFields))
    +        .getOrElse(ParquetSchemaConverter.EMPTY_MESSAGE)
    +    } else {
    +      // Spark's built-in Parquet reader will throw an exception in some cases if the requested
    +      // schema is not the same as the clipped schema
    --- End diff --
    
    I believe the failure occurs because the requested schema and file schema—while having columns with identical names and types—have columns in different order. Of the one test that fails in the `ParquetFilterSuite`, namely "Filter applied on merged Parquet schema with new column should work", it appears to be the only one for which the order of the columns is changed. These are the file and requested schema for that test:
    
    ```
    Parquet file schema:
    message spark_schema {
      required int32 c;
      optional binary b (UTF8);
    }
    
    Parquet requested schema:
    message spark_schema {
      optional binary b (UTF8);
      required int32 c;
    }
    ```
    
    I would say the Spark reader expects identical column order, whereas the parquet-mr reader accepts different column order but identical (or compatible) column names. That's my supposition at least.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #93390 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93390/testReport)** for PR 21320 at commit [`d027eef`](https://github.com/apache/spark/commit/d027eefab2b0d84f86f2a61648e646e0c8a8d3d0).


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199648692
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala ---
    @@ -182,18 +182,20 @@ private[parquet] class ParquetRowConverter(
     
       // Converters for each field.
       private val fieldConverters: Array[Converter with HasParentContainerUpdater] = {
    -    parquetType.getFields.asScala.zip(catalystType).zipWithIndex.map {
    -      case ((parquetFieldType, catalystField), ordinal) =>
    -        // Converted field value should be set to the `ordinal`-th cell of `currentRow`
    -        newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal))
    +    parquetType.getFields.asScala.map {
    +      case parquetField =>
    +        val fieldIndex = catalystType.fieldIndex(parquetField.getName)
    --- End diff --
    
    I dropped into the `sql/console` and attempted to write a parquet file with duplicate column names. It didn't work. Transcript below.
    
    ```
    scala> import org.apache.spark.sql._
    import org.apache.spark.sql._
    
    scala> val sameColumnNames = StructType(StructField("a", IntegerType) :: StructField("a", StringType) :: Nil)
    sameColumnNames: org.apache.spark.sql.types.StructType = StructType(StructField(a,IntegerType,true), StructField(a,StringType,true))
    
    scala> val rowRDD = sqlContext.sparkContext.parallelize(Row(1, "one") :: Row(2, "two") :: Nil, 1)
    rowRDD: org.apache.spark.rdd.RDD[org.apache.spark.sql.Row] = ParallelCollectionRDD[0] at parallelize at <console>:51
    
    scala> val df = sqlContext.createDataFrame(rowRDD, sameColumnNames)
    18/07/02 16:31:33 INFO SharedState: Setting hive.metastore.warehouse.dir ('null') to the value of spark.sql.warehouse.dir ('file:/Volumes/VideoAmpCS/msa/workspace/spark-public/spark-warehouse').
    18/07/02 16:31:33 INFO SharedState: Warehouse path is 'file:/Volumes/VideoAmpCS/msa/workspace/spark-public/spark-warehouse'.
    18/07/02 16:31:33 INFO ContextHandler: Started o.e.j.s.ServletContextHandler@7b13b737{/SQL,null,AVAILABLE,@Spark}
    18/07/02 16:31:33 INFO ContextHandler: Started o.e.j.s.ServletContextHandler@3c9fb104{/SQL/json,null,AVAILABLE,@Spark}
    18/07/02 16:31:33 INFO ContextHandler: Started o.e.j.s.ServletContextHandler@3d5cadbe{/SQL/execution,null,AVAILABLE,@Spark}
    18/07/02 16:31:33 INFO ContextHandler: Started o.e.j.s.ServletContextHandler@73732e26{/SQL/execution/json,null,AVAILABLE,@Spark}
    18/07/02 16:31:33 INFO ContextHandler: Started o.e.j.s.ServletContextHandler@72a13c4a{/static/sql,null,AVAILABLE,@Spark}
    18/07/02 16:31:34 INFO StateStoreCoordinatorRef: Registered StateStoreCoordinator endpoint
    df: org.apache.spark.sql.DataFrame = [a: int, a: string]
    
    scala> df.write.parquet("sameColumnNames.parquet")
    org.apache.spark.sql.AnalysisException: Found duplicate column(s) when inserting into file:/Volumes/VideoAmpCS/msa/workspace/spark-public/sameColumnNames.parquet: `a`;
      at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:85)
      at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtils.scala:42)
      at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:64)
      at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:104)
      at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:102)
      at org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:122)
      at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
      at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
      at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
      at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
      at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
      at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
      at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
      at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
      at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:662)
      at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:662)
      at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:78)
      at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:125)
      at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73)
      at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:662)
      at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:273)
      at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:267)
      at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:225)
      at org.apache.spark.sql.DataFrameWriter.parquet(DataFrameWriter.scala:554)
      ... 42 elided
    ```


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91679 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91679/testReport)** for PR 21320 at commit [`89febc8`](https://github.com/apache/spark/commit/89febc8e978d606e32911088e9589462805b8697).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by xinxin-stripe <gi...@git.apache.org>.
Github user xinxin-stripe commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Hello, we've been using your patch at Stripe and we've found something that looks like a new correctness issue:
    ```
    import spark.implicits._
    
    case class Inner(a: String)
    case class Outer(key: String, inner: Inner)
    
    val obj = Outer("key", Inner("a"))
    
    val ds = spark.createDataset(Seq(obj))
      .groupByKey(_.key)
      .reduceGroups((struct1, struct2) => struct1)
      .map(_._2)
    
    ds.collect.head shouldBe obj
    // This fails with java.lang.RuntimeException: Couldn't find inner#38 in [key#37,value#41,a#61]
    ```


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r190484386
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala ---
    @@ -99,27 +100,28 @@ trait ConstraintHelper {
       }
     
       /**
    -   * Infer the Attribute-specific IsNotNull constraints from the null intolerant child expressions
    -   * of constraints.
    +   * Infer the Attribute and ExtractValue-specific IsNotNull constraints from the null intolerant
    +   * child expressions of constraints.
        */
       private def inferIsNotNullConstraints(constraint: Expression): Seq[Expression] =
         constraint match {
           // When the root is IsNotNull, we can push IsNotNull through the child null intolerant
           // expressions
    -      case IsNotNull(expr) => scanNullIntolerantAttribute(expr).map(IsNotNull(_))
    +      case IsNotNull(expr) => scanNullIntolerantField(expr).map(IsNotNull(_))
           // Constraints always return true for all the inputs. That means, null will never be returned.
           // Thus, we can infer `IsNotNull(constraint)`, and also push IsNotNull through the child
           // null intolerant expressions.
    -      case _ => scanNullIntolerantAttribute(constraint).map(IsNotNull(_))
    +      case _ => scanNullIntolerantField(constraint).map(IsNotNull(_))
         }
     
       /**
    -   * Recursively explores the expressions which are null intolerant and returns all attributes
    -   * in these expressions.
    +   * Recursively explores the expressions which are null intolerant and returns all attributes and
    +   * complex type extractors in these expressions.
        */
    -  private def scanNullIntolerantAttribute(expr: Expression): Seq[Attribute] = expr match {
    +  private def scanNullIntolerantField(expr: Expression): Seq[Expression] = expr match {
    +    case ev: ExtractValue => Seq(ev)
    --- End diff --
    
    I agree adding more direct and independent test coverage for this change is a good idea. However, omitting this change will weaken the capabilities of this PR. It would also imply the removal of the failing test case in `ParquetSchemaPruningSuite`, which would imply two follow on PRs. The first would be to add this specific change plus the right test coverage. The next would be to restore the test case removed from 'ParquetSchemaPruningSuite'.
    
    Let me suggest an alternative. As this change is a valuable enhancement for this PR, let me try adding an appropriate test case in `InferFiltersFromConstraintsSuite` as part of this PR. That will eliminate the requirement for two more follow-on PRs.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mswit-databricks <gi...@git.apache.org>.
Github user mswit-databricks commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199415439
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -71,9 +80,22 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
           StructType.fromString(schemaString)
         }
     
    -    val parquetRequestedSchema =
    +    val clippedParquetSchema =
           ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema)
     
    +    val parquetRequestedSchema = if (parquetMrCompatibility) {
    +      // Parquet-mr will throw an exception if we try to read a superset of the file's schema.
    +      // Therefore, we intersect our clipped schema with the underlying file's schema
    +      ParquetReadSupport.intersectParquetGroups(clippedParquetSchema, context.getFileSchema)
    +        .map(intersectionGroup =>
    +          new MessageType(intersectionGroup.getName, intersectionGroup.getFields))
    +        .getOrElse(ParquetSchemaConverter.EMPTY_MESSAGE)
    +    } else {
    +      // Spark's built-in Parquet reader will throw an exception in some cases if the requested
    +      // schema is not the same as the clipped schema
    --- End diff --
    
    CC @michal-databricks 


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r204208518
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala ---
    @@ -0,0 +1,387 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.scalatest.BeforeAndAfterAll
    +import org.scalatest.exceptions.TestFailedException
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.dsl.plans._
    +import org.apache.spark.sql.catalyst.expressions.NamedExpression
    +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
    +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
    +import org.apache.spark.sql.types._
    +
    +// scalastyle:off line.size.limit
    +class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll {
    +  // The test schema as a tree string, i.e. `schema.treeString`
    +  // root
    +  //  |-- col1: string (nullable = false)
    +  //  |-- col2: struct (nullable = true)
    +  //  |    |-- field1: integer (nullable = true)
    +  //  |    |-- field2: array (nullable = true)
    +  //  |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field3: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield3: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = true)
    +  //  |    |-- field4: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field5: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: struct (nullable = false)
    +  //  |    |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: struct (nullable = true)
    +  //  |    |    |    |    |-- subsubfield1: struct (nullable = true)
    +  //  |    |    |    |    |    |-- subsubsubfield1: string (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field6: struct (nullable = true)
    +  //  |    |    |-- subfield1: string (nullable = false)
    +  //  |    |    |-- subfield2: string (nullable = true)
    +  //  |    |-- field7: struct (nullable = true)
    +  //  |    |    |-- subfield1: struct (nullable = true)
    +  //  |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field8: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: array (valueContainsNull = false)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field9: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col3: array (nullable = false)
    +  //  |    |-- element: struct (containsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col4: map (nullable = false)
    +  //  |    |-- key: string
    +  //  |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col5: array (nullable = true)
    +  //  |    |-- element: map (containsNull = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col6: map (nullable = true)
    +  //  |    |-- key: string
    +  //  |    |-- value: array (valueContainsNull = true)
    +  //  |    |    |-- element: struct (containsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col7: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: integer (nullable = false)
    +  //  |    |    |-- field2: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |-- field3: array (nullable = true)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |-- col8: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: array (nullable = false)
    +  //  |    |    |    |-- element: integer (containsNull = false)
    +  private val schema =
    +    StructType(
    +      StructField("col1", StringType, nullable = false) ::
    +      StructField("col2", StructType(
    +        StructField("field1", IntegerType) ::
    +        StructField("field2", ArrayType(IntegerType, containsNull = false)) ::
    +        StructField("field3", ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", IntegerType) ::
    +          StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) ::
    +        StructField("field4", MapType(StringType, StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) ::
    +        StructField("field5", ArrayType(StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil), nullable = false) ::
    +          StructField("subfield2", StructType(
    +            StructField("subsubfield1", StructType(
    +              StructField("subsubsubfield1", StringType) :: Nil)) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)), nullable = false) ::
    +        StructField("field6", StructType(
    +          StructField("subfield1", StringType, nullable = false) ::
    +          StructField("subfield2", StringType) :: Nil)) ::
    +        StructField("field7", StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) ::
    +        StructField("field8", MapType(StringType, ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil)), valueContainsNull = false)) ::
    +        StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) ::
    +     StructField("col3", ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) ::
    +     StructField("col4", MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) ::
    +     StructField("col5", ArrayType(MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) ::
    +     StructField("col6", MapType(StringType, ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) ::
    +     StructField("col7", ArrayType(StructType(
    +       StructField("field1", IntegerType, nullable = false) ::
    +       StructField("field2", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil)) ::
    +       StructField("field3", ArrayType(StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) ::
    +     StructField("col8", ArrayType(StructType(
    +       StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) :: Nil)
    +
    +  private val testRelation = LocalRelation(schema.toAttributes)
    +
    +  test("SelectedField should not match an attribute reference") {
    +    assertResult(None)(unapplySelect("col1"))
    +    assertResult(None)(unapplySelect("col1 as foo"))
    +    assertResult(None)(unapplySelect("col2"))
    +  }
    +
    +  info("For a relation with schema\n" + indent(schema.treeString))
    +
    +  testSelect("col2.field2", "col2.field2[0] as foo") {
    +    StructField("col2", StructType(
    +      StructField("field2", ArrayType(IntegerType, containsNull = false)) :: Nil))
    +  }
    +
    +  testSelect("col2.field9", "col2.field9['foo'] as foo") {
    +    StructField("col2", StructType(
    +      StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil))
    +  }
    +
    +  testSelect("col2.field3.subfield3", "col2.field3[0].subfield3 as foo",
    +      "col2.field3.subfield3[0] as foo", "col2.field3[0].subfield3[0] as foo") {
    +    StructField("col2", StructType(
    +      StructField("field3", ArrayType(StructType(
    +        StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) :: Nil))
    +  }
    +
    +  testSelect("col2.field3.subfield1") {
    +    StructField("col2", StructType(
    +      StructField("field3", ArrayType(StructType(
    +        StructField("subfield1", IntegerType) :: Nil)), nullable = false) :: Nil))
    +  }
    +
    +  testSelect("col2.field5.subfield1") {
    +    StructField("col2", StructType(
    +      StructField("field5", ArrayType(StructType(
    +        StructField("subfield1", StructType(
    +          StructField("subsubfield1", IntegerType) ::
    +          StructField("subsubfield2", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil))
    +  }
    +
    +  testSelect("col3.field1.subfield1") {
    +    StructField("col3", ArrayType(StructType(
    +      StructField("field1", StructType(
    +        StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), containsNull = false), nullable = false)
    +  }
    +
    +  testSelect("col3.field2['foo'] as foo") {
    +    StructField("col3", ArrayType(StructType(
    +      StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false)
    +  }
    +
    +  testSelect("col4['foo'].field1.subfield1 as foo") {
    +    StructField("col4", MapType(StringType, StructType(
    +      StructField("field1", StructType(
    +        StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil), valueContainsNull = false), nullable = false)
    +  }
    +
    +  testSelect("col4['foo'].field2['bar'] as foo") {
    +    StructField("col4", MapType(StringType, StructType(
    +      StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false)
    +  }
    +
    +  testSelect("col5[0]['foo'].field1.subfield1 as foo") {
    +    StructField("col5", ArrayType(MapType(StringType, StructType(
    +      StructField("field1", StructType(
    +        StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false)))
    +  }
    +
    +  testSelect("col6['foo'][0].field1.subfield1 as foo") {
    +    StructField("col6", MapType(StringType, ArrayType(StructType(
    +      StructField("field1", StructType(
    +        StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false)))
    +  }
    +
    +  testSelect("col2.field5.subfield1.subsubfield1") {
    +    StructField("col2", StructType(
    +      StructField("field5", ArrayType(StructType(
    +        StructField("subfield1", StructType(
    +          StructField("subsubfield1", IntegerType) :: Nil), nullable = false) :: Nil)), nullable = false) :: Nil))
    +  }
    +
    +  testSelect("col2.field5.subfield2.subsubfield1.subsubsubfield1") {
    +    StructField("col2", StructType(
    +      StructField("field5", ArrayType(StructType(
    +        StructField("subfield2", StructType(
    +          StructField("subsubfield1", StructType(
    +            StructField("subsubsubfield1", StringType) :: Nil)) :: Nil)) :: Nil)), nullable = false) :: Nil))
    +  }
    +
    +  testSelect("col2.field4['foo'].subfield1 as foo") {
    +    StructField("col2", StructType(
    +      StructField("field4", MapType(StringType, StructType(
    +        StructField("subfield1", IntegerType) :: Nil), valueContainsNull = false)) :: Nil))
    +  }
    +
    +  testSelect("col2.field4['foo'].subfield2 as foo", "col2.field4['foo'].subfield2[0] as foo") {
    +    StructField("col2", StructType(
    +      StructField("field4", MapType(StringType, StructType(
    +        StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) :: Nil))
    +  }
    +
    +  testSelect("col2.field8['foo'][0].subfield1 as foo") {
    +    StructField("col2", StructType(
    +      StructField("field8", MapType(StringType, ArrayType(StructType(
    +        StructField("subfield1", IntegerType) :: Nil)), valueContainsNull = false)) :: Nil))
    +  }
    +
    +  testSelect("col2.field1") {
    +    StructField("col2", StructType(
    +      StructField("field1", IntegerType) :: Nil))
    +  }
    +
    +  testSelect("col2.field6") {
    +    StructField("col2", StructType(
    +      StructField("field6", StructType(
    +        StructField("subfield1", StringType, nullable = false) ::
    +        StructField("subfield2", StringType) :: Nil)) :: Nil))
    +  }
    +
    +  testSelect("col2.field7.subfield1") {
    +    StructField("col2", StructType(
    +      StructField("field7", StructType(
    +        StructField("subfield1", StructType(
    +          StructField("subsubfield1", IntegerType) ::
    +          StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) :: Nil))
    +  }
    +
    +  testSelect("col2.field6.subfield1") {
    +    StructField("col2", StructType(
    +      StructField("field6", StructType(
    +        StructField("subfield1", StringType, nullable = false) :: Nil)) :: Nil))
    +  }
    +
    +  testSelect("col7.field1", "col7[0].field1 as foo", "col7.field1[0] as foo") {
    +    StructField("col7", ArrayType(StructType(
    +      StructField("field1", IntegerType, nullable = false) :: Nil)))
    +  }
    +
    +  testSelect("col7.field2.subfield1") {
    +    StructField("col7", ArrayType(StructType(
    +      StructField("field2", StructType(
    +        StructField("subfield1", IntegerType, nullable = false) :: Nil)) :: Nil)))
    +  }
    +
    +  testSelect("col7.field3.subfield1") {
    +    StructField("col7", ArrayType(StructType(
    +      StructField("field3", ArrayType(StructType(
    +        StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil)))
    +  }
    +
    +  testSelect("col8.field1", "col8[0].field1 as foo", "col8.field1[0] as foo", "col8[0].field1[0] as foo") {
    +    StructField("col8", ArrayType(StructType(
    +      StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil)))
    +  }
    +
    +  def assertResult(expected: StructField)(actual: StructField)(selectExpr: String): Unit = {
    +    try {
    +      super.assertResult(expected)(actual)
    +    } catch {
    +      case ex: TestFailedException =>
    +        // Print some helpful diagnostics in the case of failure
    +        alert("Expected SELECT \"" + selectExpr + "\" to select the schema\n" +
    +          indent(StructType(expected :: Nil).treeString) +
    +          indent("but it actually selected\n") +
    +          indent(StructType(actual :: Nil).treeString) +
    +          indent("Note that expected.dataType.sameType(actual.dataType) = " +
    +          expected.dataType.sameType(actual.dataType)))
    +        throw ex
    +    }
    +  }
    +
    +  // Test that the given SELECT expressions prune the test schema to the single-column schema
    +  // defined by the given field
    +  private def testSelect(selectExpr: String, otherSelectExprs: String*)(expected: StructField) {
    +    val selectExprs = selectExpr +: otherSelectExprs
    +    test(s"SELECT " + selectExprs.map(s => s""""$s"""").mkString(", ") + " should select the schema\n" +
    +        indent(StructType(expected :: Nil).treeString)) {
    +      for (selectExpr <- selectExprs) {
    +        assertSelect(selectExpr, expected)
    +      }
    +    }
    +  }
    +
    +  private def assertSelect(expr: String, expected: StructField) = {
    +    unapplySelect(expr) match {
    +      case Some(field) =>
    +        assertResult(expected)(field)(expr)
    +      case None =>
    +        val failureMessage =
    +          "Failed to select a field from " + expr + ". " +
    +          "Expected:\n" +
    +          StructType(expected :: Nil).treeString
    +        fail(failureMessage)
    +    }
    +  }
    +
    +  private def unapplySelect(expr: String) = {
    +    val parsedExpr =
    +      CatalystSqlParser.parseExpression(expr) match {
    +        case namedExpr: NamedExpression => namedExpr
    +      }
    +    val select = testRelation.select(parsedExpr)
    +    val analyzed = select.analyze
    +    SelectedField.unapply(analyzed.expressions.head)
    +  }
    +
    +  private def indent(string: String) = string.replaceAll("(?m)^", "   ")
    --- End diff --
    
    This method indents every line in a string by four spaces. For example, https://regex101.com/r/teuQcA/1.
    
    I'll add an explanatory comment in my next push.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #90582 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90582/testReport)** for PR 21320 at commit [`9e301b3`](https://github.com/apache/spark/commit/9e301b37bb5863ef5fad8ec15f1d8f3d4b5e6a6f).


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r190486220
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -1256,8 +1256,18 @@ object SQLConf {
             "issues. Turn on this config to insert a local sort before actually doing repartition " +
             "to generate consistent repartition results. The performance of repartition() may go " +
             "down since we insert extra local sort before it.")
    +        .booleanConf
    +        .createWithDefault(true)
    +
    +  val NESTED_SCHEMA_PRUNING_ENABLED =
    +    buildConf("spark.sql.nestedSchemaPruning.enabled")
    +      .internal()
    +      .doc("Prune nested fields from a logical relation's output which are unnecessary in " +
    +        "satisfying a query. This optimization allows columnar file format readers to avoid " +
    +        "reading unnecessary nested column data. Currently Parquet is the only data source that " +
    --- End diff --
    
    ORC should be able to support this capability as well, but this PR does not address that.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3923/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by dbtsai <gi...@git.apache.org>.
Github user dbtsai commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Thanks @mallman for schema pruning work which will be a big win in our pattern of accessing our data.
    
    I'm testing this new feature, and find `where clause` on the selected nested column can break the schema pruning.
    
    For example,
    ```scala
        val q1 = sql("select name.first from contacts")
        val q2 = sql("select name.first from contacts where name.first = 'David'")
    
        q1.explain(true)
        q2.explain(true)
    ```
    
    The physical plan of `q1` is right and what we expect with this feature,
    ```
    == Physical Plan ==
    *(1) Project [name#19.first AS first#36]
    +- *(1) FileScan parquet [name#19] Batched: false, Format: Parquet, PartitionFilters: [], 
        PushedFilters: [], ReadSchema: struct<name:struct<first:string>>
    ```
    
    But the physical plan of `q2` will have a pushed filter on `name` resulting reading the entire `name` column,
    ```
    == Physical Plan ==
    *(1) Project [name#19.first AS first#40]
    +- *(1) Filter (isnotnull(name#19) && (name#19.first = David))
       +- *(1) FileScan parquet [name#19] Batched: false, Format: Parquet, PartitionFilters: [], 
        PushedFilters: [IsNotNull(name)], ReadSchema: struct<name:struct<first:string,middle:string,last:string>>
    ```
    
    I understand that predicate push-down on the nested column is not implemented yet, and with schema pruning and `where clause`, we should be able to only read the selected nested columns and the columns with `where caluse`.
    
    Thanks.
    
    cc @beettlle


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Try this when `spark.sql.nestedSchemaPruning.enabled` is on?
    ```SQL
        withTable("t1") {
          spark.sql(
            """
              |Create table t1 (`id` INT,`CoL1` STRING,
              |`coL2` STRUCT<`a`: TIMESTAMP, `b`: INT, `c`: ARRAY<DATE>>,
              |`col3` INT,
              |`col4` INT,
              |`Col5` STRUCT<`a`: STRING, `b`: STRUCT<`a`: INT, `b`: ARRAY<INT>, `c`: INT>, `c`: INT>)
              |USING parquet
            """.stripMargin)
          spark.sql("SELECT id from t1 where col2.b < 7").count()
        }
    ```
    



---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94969/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    LGTM, as I explained above. https://github.com/apache/spark/pull/21320#issuecomment-415526369
    
    Thanks for your patience and great work! @mallman Sorry, it takes two years to merge the code. 
    



---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Hello, we've been using your patch at Stripe and we've found something that looks like a new bug:
    
    Thank you for sharing this, @xinxin-stripe. This is very helpful. I will investigate and report back.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r212388958
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala ---
    @@ -202,11 +204,15 @@ private[parquet] class ParquetRowConverter(
     
       override def start(): Unit = {
         var i = 0
    -    while (i < currentRow.numFields) {
    +    while (i < fieldConverters.length) {
           fieldConverters(i).updater.start()
           currentRow.setNullAt(i)
           i += 1
         }
    +    while (i < currentRow.numFields) {
    --- End diff --
    
    I'll get back to you on this shortly. Thanks.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205021712
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala ---
    @@ -0,0 +1,388 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.scalatest.BeforeAndAfterAll
    +import org.scalatest.exceptions.TestFailedException
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.dsl.plans._
    +import org.apache.spark.sql.catalyst.expressions.NamedExpression
    +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
    +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
    +import org.apache.spark.sql.types._
    +
    +// scalastyle:off line.size.limit
    --- End diff --
    
    No comment.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > I have no intention to at this point, no.
    
    Yup, but I guess we should do when we are about to be complete to avoid breaking things by switching this feature on.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by IgorBerman <gi...@git.apache.org>.
Github user IgorBerman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman, wanted to say huge thanks for your work! this is great step forward.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Then should we keep this one or #21889? shall we deduplicate the efforts? I requested to open that because this looks going to be inactive per your comments.
    
    As I stated before, I'll continue pushing changes to this branch. However, the window of opportunity to review syntax and style in this PR closed long ago. If someone wants to put forward that kind of comment for review I will consider it at my discretion. I'm not going to guarantee action or even a response. If someone relays a bug or a concern regarding correctness or performance, I will address it.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Add some test cases when turning on `spark.sql.caseSensitive`?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r201863463
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/SelectedField.scala ---
    @@ -0,0 +1,134 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * A Scala extractor that builds a [[org.apache.spark.sql.types.StructField]] from a Catalyst
    + * complex type extractor. For example, consider a relation with the following schema:
    + *
    + *   {{{
    + *   root
    + *    |-- name: struct (nullable = true)
    + *    |    |-- first: string (nullable = true)
    + *    |    |-- last: string (nullable = true)
    + *    }}}
    + *
    + * Further, suppose we take the select expression `name.first`. This will parse into an
    + * `Alias(child, "first")`. Ignoring the alias, `child` matches the following pattern:
    + *
    + *   {{{
    + *   GetStructFieldObject(
    + *     AttributeReference("name", StructType(_), _, _),
    + *     StructField("first", StringType, _, _))
    + *   }}}
    + *
    + * [[SelectedField]] converts that expression into
    + *
    + *   {{{
    + *   StructField("name", StructType(Array(StructField("first", StringType))))
    + *   }}}
    + *
    + * by mapping each complex type extractor to a [[org.apache.spark.sql.types.StructField]] with the
    + * same name as its child (or "parent" going right to left in the select expression) and a data
    + * type appropriate to the complex type extractor. In our example, the name of the child expression
    + * is "name" and its data type is a [[org.apache.spark.sql.types.StructType]] with a single string
    + * field named "first".
    + *
    + * @param expr the top-level complex type extractor
    + */
    +object SelectedField {
    +  def unapply(expr: Expression): Option[StructField] = {
    --- End diff --
    
    The code does not compile with that change.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r204206072
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ---
    @@ -0,0 +1,156 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import java.io.File
    +
    +import org.apache.spark.sql.{QueryTest, Row}
    +import org.apache.spark.sql.execution.FileSchemaPruningTest
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class ParquetSchemaPruningSuite
    +    extends QueryTest
    +    with ParquetTest
    +    with FileSchemaPruningTest
    +    with SharedSQLContext {
    +  case class FullName(first: String, middle: String, last: String)
    +  case class Contact(name: FullName, address: String, pets: Int, friends: Array[FullName] = Array(),
    +    relatives: Map[String, FullName] = Map())
    +
    +  val contacts =
    +    Contact(FullName("Jane", "X.", "Doe"), "123 Main Street", 1) ::
    +    Contact(FullName("John", "Y.", "Doe"), "321 Wall Street", 3) :: Nil
    +
    +  case class Name(first: String, last: String)
    +  case class BriefContact(name: Name, address: String)
    +
    +  val briefContacts =
    +    BriefContact(Name("Janet", "Jones"), "567 Maple Drive") ::
    +    BriefContact(Name("Jim", "Jones"), "6242 Ash Street") :: Nil
    +
    +  case class ContactWithDataPartitionColumn(name: FullName, address: String, pets: Int,
    +    friends: Array[FullName] = Array(), relatives: Map[String, FullName] = Map(), p: Int)
    +
    +  case class BriefContactWithDataPartitionColumn(name: Name, address: String, p: Int)
    +
    +  val contactsWithDataPartitionColumn =
    +    contacts.map { case Contact(name, address, pets, friends, relatives) =>
    +      ContactWithDataPartitionColumn(name, address, pets, friends, relatives, 1) }
    +  val briefContactsWithDataPartitionColumn =
    +    briefContacts.map { case BriefContact(name: Name, address: String) =>
    +      BriefContactWithDataPartitionColumn(name, address, 2) }
    +
    +  testSchemaPruning("select a single complex field") {
    +    val query = sql("select name.middle from contacts")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query, Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil)
    +  }
    +
    +  testSchemaPruning("select a single complex field and the partition column") {
    +    val query = sql("select name.middle, p from contacts")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query, Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil)
    +  }
    +
    +  testSchemaPruning("partial schema intersection - select missing subfield") {
    +    val query = sql("select name.middle, address from contacts where p=2")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>,address:string>")
    +    checkAnswer(query,
    +      Row(null, "567 Maple Drive") ::
    +      Row(null, "6242 Ash Street") :: Nil)
    +  }
    +
    +  testSchemaPruning("no unnecessary schema pruning") {
    +    val query =
    +      sql("select name.last, name.middle, name.first, relatives[''].last, " +
    +        "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " +
    +        "friends[0].first, pets, address from contacts where p=2")
    +    // We've selected every field in the schema. Therefore, no schema pruning should be performed.
    +    // We check this by asserting that the scanned schema of the query is identical to the schema
    +    // of the contacts relation, even though the fields are selected in different orders.
    +    checkScanSchemata(query,
    +      "struct<name:struct<first:string,middle:string,last:string>,address:string,pets:int," +
    +      "friends:array<struct<first:string,middle:string,last:string>>," +
    +      "relatives:map<string,struct<first:string,middle:string,last:string>>>")
    +    checkAnswer(query,
    +      Row("Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") ::
    +      Row("Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") ::
    +      Nil)
    +  }
    +
    +  testSchemaPruning("empty schema intersection") {
    +    val query = sql("select name.middle from contacts where p=2")
    +    checkScanSchemata(query, "struct<name:struct<middle:string>>")
    +    checkAnswer(query,
    +      Row(null) :: Row(null) :: Nil)
    +  }
    +
    +  private def testSchemaPruning(testName: String)(testThunk: => Unit) {
    +    withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> "false") {
    --- End diff --
    
    Whoa. I just noticed I've been switching on/off the wrong configuration option in my tests. I've meant to be testing with and without the vectorized reader. So I should be using `PARQUET_VECTORIZED_READER_ENABLED`, right?
    
    Also, just to make sure, how many parquet reader implementations does Spark have? I think it's two. There's the one from the parquet-mr library, and there's another (vectorized) one that's part of the Spark codebase. If I'm wrong about that please alert me, because I'm making some erroneous assumptions.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman, can you close this and put some efforts there in https://github.com/apache/spark/pull/21889? I see no point of leaving this PR open.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman, can you close this one and put some changes into @ajacques branch then? no point of opening duplicated changes. Since @ajacques is at least willing to address other comments including styles in https://github.com/apache/spark/pull/21889, we will go with that PR in any event. Right?
    
    If you are willing to update this PR, please address style comments here too. Otherwise, let me just close this and deduplicate the effort. What's the point of leaving this PR open?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > @mallman Could you remove the changes made in ParquetRowConverter.scala and also turn off spark.sql.nestedSchemaPruning.enabled by default in this PR?
    
    Done.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #92070 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92070/testReport)** for PR 21320 at commit [`a255bcb`](https://github.com/apache/spark/commit/a255bcb4c480d3c97f7ff0590bca0c20de034a31).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94971 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94971/testReport)** for PR 21320 at commit [`61c7937`](https://github.com/apache/spark/commit/61c7937e39fce02d9ae7385d9adaaf7bc913b7ed).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Regarding #21320 (comment), can you at least set this enable by default and see if some existing tests are broken or not?
    
    I have no intention to at this point, no.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205021140
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ---
    @@ -0,0 +1,156 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import java.io.File
    +
    +import org.apache.spark.sql.{QueryTest, Row}
    +import org.apache.spark.sql.execution.FileSchemaPruningTest
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +class ParquetSchemaPruningSuite
    +    extends QueryTest
    +    with ParquetTest
    +    with FileSchemaPruningTest
    +    with SharedSQLContext {
    --- End diff --
    
    No comment.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman Thanks!


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91443 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91443/testReport)** for PR 21320 at commit [`0351094`](https://github.com/apache/spark/commit/0351094317aa9093fd4039320f1e6e3bba4c0dc5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `  case class ContactWithDataPartitionColumn(name: FullName, address: String, pets: Int,`
      * `  case class BriefContactWithDataPartitionColumn(name: Name, address: String, p: Int)`


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #93384 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93384/testReport)** for PR 21320 at commit [`d027eef`](https://github.com/apache/spark/commit/d027eefab2b0d84f86f2a61648e646e0c8a8d3d0).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93715/
    Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91684 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91684/testReport)** for PR 21320 at commit [`7f67ec0`](https://github.com/apache/spark/commit/7f67ec0a82dd09dd867d5882dda0965fcab28974).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #92191 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92191/testReport)** for PR 21320 at commit [`a255bcb`](https://github.com/apache/spark/commit/a255bcb4c480d3c97f7ff0590bca0c20de034a31).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94476/
    Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #95041 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95041/testReport)** for PR 21320 at commit [`2711746`](https://github.com/apache/spark/commit/271174637f35aa5684ec6cc1938c4c8b210553d3).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94521 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94521/testReport)** for PR 21320 at commit [`0e5594b`](https://github.com/apache/spark/commit/0e5594b6ac1dcb94f3f0166e66a7d4e7eae3d00c).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #93384 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93384/testReport)** for PR 21320 at commit [`d027eef`](https://github.com/apache/spark/commit/d027eefab2b0d84f86f2a61648e646e0c8a8d3d0).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    retest this please



---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1985/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r201863251
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -71,9 +80,22 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
           StructType.fromString(schemaString)
         }
     
    -    val parquetRequestedSchema =
    +    val clippedParquetSchema =
           ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema)
     
    +    val parquetRequestedSchema = if (parquetMrCompatibility) {
    +      // Parquet-mr will throw an exception if we try to read a superset of the file's schema.
    +      // Therefore, we intersect our clipped schema with the underlying file's schema
    +      ParquetReadSupport.intersectParquetGroups(clippedParquetSchema, context.getFileSchema)
    +        .map(intersectionGroup =>
    +          new MessageType(intersectionGroup.getName, intersectionGroup.getFields))
    +        .getOrElse(ParquetSchemaConverter.EMPTY_MESSAGE)
    +    } else {
    +      // Spark's built-in Parquet reader will throw an exception in some cases if the requested
    +      // schema is not the same as the clipped schema
    --- End diff --
    
    @gatorsmile @rdblue @mswit-databricks What is your position on this? I don't know that the parquet spec provides a definitive answer on this question.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    gentle ping @mallman since the code freeze is close


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Retest this please.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #92191 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92191/testReport)** for PR 21320 at commit [`a255bcb`](https://github.com/apache/spark/commit/a255bcb4c480d3c97f7ff0590bca0c20de034a31).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #4277 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4277/testReport)** for PR 21320 at commit [`0e5594b`](https://github.com/apache/spark/commit/0e5594b6ac1dcb94f3f0166e66a7d4e7eae3d00c).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2330/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r189492534
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/planning/SelectedFieldSuite.scala ---
    @@ -0,0 +1,432 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.scalatest.BeforeAndAfterAll
    +import org.scalatest.exceptions.TestFailedException
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.dsl.plans._
    +import org.apache.spark.sql.catalyst.expressions.NamedExpression
    +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
    +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
    +import org.apache.spark.sql.types._
    +
    +// scalastyle:off line.size.limit
    +class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll {
    +  // The test schema as a tree string, i.e. `schema.treeString`
    +  // root
    +  //  |-- col1: string (nullable = false)
    +  //  |-- col2: struct (nullable = true)
    +  //  |    |-- field1: integer (nullable = true)
    +  //  |    |-- field2: array (nullable = true)
    +  //  |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field3: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield3: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = true)
    +  //  |    |-- field4: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field5: array (nullable = false)
    +  //  |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |-- subfield1: struct (nullable = false)
    +  //  |    |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |    |    |-- subfield2: struct (nullable = true)
    +  //  |    |    |    |    |-- subsubfield1: struct (nullable = true)
    +  //  |    |    |    |    |    |-- subsubsubfield1: string (nullable = true)
    +  //  |    |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field6: struct (nullable = true)
    +  //  |    |    |-- subfield1: string (nullable = false)
    +  //  |    |    |-- subfield2: string (nullable = true)
    +  //  |    |-- field7: struct (nullable = true)
    +  //  |    |    |-- subfield1: struct (nullable = true)
    +  //  |    |    |    |-- subsubfield1: integer (nullable = true)
    +  //  |    |    |    |-- subsubfield2: integer (nullable = true)
    +  //  |    |-- field8: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: array (valueContainsNull = false)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: array (nullable = true)
    +  //  |    |    |    |    |    |-- element: integer (containsNull = false)
    +  //  |    |-- field9: map (nullable = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col3: array (nullable = false)
    +  //  |    |-- element: struct (containsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col4: map (nullable = false)
    +  //  |    |-- key: string
    +  //  |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |    |    |-- field2: map (nullable = true)
    +  //  |    |    |    |-- key: string
    +  //  |    |    |    |-- value: integer (valueContainsNull = false)
    +  //  |-- col5: array (nullable = true)
    +  //  |    |-- element: map (containsNull = true)
    +  //  |    |    |-- key: string
    +  //  |    |    |-- value: struct (valueContainsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col6: map (nullable = true)
    +  //  |    |-- key: string
    +  //  |    |-- value: array (valueContainsNull = true)
    +  //  |    |    |-- element: struct (containsNull = false)
    +  //  |    |    |    |-- field1: struct (nullable = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = true)
    +  //  |    |    |    |    |-- subfield2: integer (nullable = true)
    +  //  |-- col7: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: integer (nullable = false)
    +  //  |    |    |-- field2: struct (nullable = true)
    +  //  |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |    |    |-- field3: array (nullable = true)
    +  //  |    |    |    |-- element: struct (containsNull = true)
    +  //  |    |    |    |    |-- subfield1: integer (nullable = false)
    +  //  |-- col8: array (nullable = true)
    +  //  |    |-- element: struct (containsNull = true)
    +  //  |    |    |-- field1: array (nullable = false)
    +  //  |    |    |    |-- element: integer (containsNull = false)
    +  private val schema =
    +    StructType(
    +      StructField("col1", StringType, nullable = false) ::
    +      StructField("col2", StructType(
    +        StructField("field1", IntegerType) ::
    +        StructField("field2", ArrayType(IntegerType, containsNull = false)) ::
    +        StructField("field3", ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", IntegerType) ::
    +          StructField("subfield3", ArrayType(IntegerType)) :: Nil)), nullable = false) ::
    +        StructField("field4", MapType(StringType, StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil), valueContainsNull = false)) ::
    +        StructField("field5", ArrayType(StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil), nullable = false) ::
    +          StructField("subfield2", StructType(
    +            StructField("subsubfield1", StructType(
    +              StructField("subsubsubfield1", StringType) :: Nil)) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)), nullable = false) ::
    +        StructField("field6", StructType(
    +          StructField("subfield1", StringType, nullable = false) ::
    +          StructField("subfield2", StringType) :: Nil)) ::
    +        StructField("field7", StructType(
    +          StructField("subfield1", StructType(
    +            StructField("subsubfield1", IntegerType) ::
    +            StructField("subsubfield2", IntegerType) :: Nil)) :: Nil)) ::
    +        StructField("field8", MapType(StringType, ArrayType(StructType(
    +          StructField("subfield1", IntegerType) ::
    +          StructField("subfield2", ArrayType(IntegerType, containsNull = false)) :: Nil)), valueContainsNull = false)) ::
    +        StructField("field9", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil)) ::
    +     StructField("col3", ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), containsNull = false), nullable = false) ::
    +     StructField("col4", MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) ::
    +         StructField("subfield2", IntegerType) :: Nil)) ::
    +       StructField("field2", MapType(StringType, IntegerType, valueContainsNull = false)) :: Nil), valueContainsNull = false), nullable = false) ::
    +     StructField("col5", ArrayType(MapType(StringType, StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) ::
    +     StructField("col6", MapType(StringType, ArrayType(StructType(
    +       StructField("field1", StructType(
    +         StructField("subfield1", IntegerType) ::
    +         StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) ::
    +     StructField("col7", ArrayType(StructType(
    +       StructField("field1", IntegerType, nullable = false) ::
    +       StructField("field2", StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil)) ::
    +       StructField("field3", ArrayType(StructType(
    +         StructField("subfield1", IntegerType, nullable = false) :: Nil))) :: Nil))) ::
    +     StructField("col8", ArrayType(StructType(
    +       StructField("field1", ArrayType(IntegerType, containsNull = false), nullable = false) :: Nil))) :: Nil)
    +
    +  private val testRelation = LocalRelation(schema.toAttributes)
    +
    +  test("should not match an attribute reference") {
    +    assertResult(None)(unapplySelect("col1"))
    +    assertResult(None)(unapplySelect("col1 as foo"))
    +    assertResult(None)(unapplySelect("col2"))
    +  }
    +
    +  test("col2.field2, col2.field2[0] as foo") {
    --- End diff --
    
    Could you change the test case names based on the goal of these tests? The reviewers and the future coders/readers can easily find out whether these tests cover all the data types and scenarios.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2332/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2003/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r204288233
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionRootFields = projects.flatMap(getRootFields)
    +        val filterRootFields = filters.flatMap(getRootFields)
    +        val requestedRootFields = (projectionRootFields ++ filterRootFields).distinct
    +
    +        // If [[requestedRootFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) {
    +          val prunedSchema = requestedRootFields
    +            .map { case RootField(field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    +          val prunedDataSchema =
    +            StructType(prunedSchema.filter(f => dataSchemaFieldNames.contains(f.name)))
    +
    +          // If the data schema is different from the pruned data schema, continue. Otherwise,
    +          // return [[op]]. We effect this comparison by counting the number of "leaf" fields in
    +          // each schemata, assuming the fields in [[prunedDataSchema]] are a subset of the fields
    +          // in [[dataSchema]].
    +          if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) {
    +            val prunedParquetRelation =
    +              hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession)
    +
    +            // We need to replace the expression ids of the pruned relation output attributes
    +            // with the expression ids of the original relation output attributes so that
    +            // references to the original relation's output are not broken
    +            val outputIdMap = l.output.map(att => (att.name, att.exprId)).toMap
    +            val prunedRelationOutput =
    +              prunedParquetRelation
    +                .schema
    +                .toAttributes
    +                .map {
    +                  case att if outputIdMap.contains(att.name) =>
    +                    att.withExprId(outputIdMap(att.name))
    +                  case att => att
    +                }
    +            val prunedRelation =
    +              l.copy(relation = prunedParquetRelation, output = prunedRelationOutput)
    +
    +            val projectionOverSchema = ProjectionOverSchema(prunedDataSchema)
    +
    +            // Construct a new target for our projection by rewriting and
    +            // including the original filters where available
    +            val projectionChild =
    +              if (filters.nonEmpty) {
    +                val projectedFilters = filters.map(_.transformDown {
    +                  case projectionOverSchema(expr) => expr
    +                })
    +                val newFilterCondition = projectedFilters.reduce(And)
    +                Filter(newFilterCondition, prunedRelation)
    +              } else {
    +                prunedRelation
    +              }
    +
    +            // Construct the new projections of our [[Project]] by
    --- End diff --
    
    tiny nit: for inlined comment we could just better write `Project` or `` `Project` ``.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by jainaks <gi...@git.apache.org>.
Github user jainaks commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman It does work fine with "name.First".


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #93715 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93715/testReport)** for PR 21320 at commit [`5decbef`](https://github.com/apache/spark/commit/5decbefbd30a46e4fcf92863eff263fa75a97b78).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Thanks @jainaks for the sample file and instructions to reproduce the problem. I will investigate and reply.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3791/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93397/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r212476400
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ---
    @@ -31,6 +32,7 @@ import org.apache.spark.sql.types.StructType
     class ParquetSchemaPruningSuite
         extends QueryTest
         with ParquetTest
    +    with SchemaPruningTest
    --- End diff --
    
    In my local, all of them still can pass. Am I wrong?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94521 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94521/testReport)** for PR 21320 at commit [`0e5594b`](https://github.com/apache/spark/commit/0e5594b6ac1dcb94f3f0166e66a7d4e7eae3d00c).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/30/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94235 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94235/testReport)** for PR 21320 at commit [`30831a6`](https://github.com/apache/spark/commit/30831a6234b1e4b455dd44eb75c9b54dedff4505).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Try this when spark.sql.nestedSchemaPruning.enabled is on?
    
    This is a case-sensitivity issue (obviously). I'll get to the root of it. Thanks.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94915/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199389252
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala ---
    @@ -71,9 +80,22 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone])
           StructType.fromString(schemaString)
         }
     
    -    val parquetRequestedSchema =
    +    val clippedParquetSchema =
           ParquetReadSupport.clipParquetSchema(context.getFileSchema, catalystRequestedSchema)
     
    +    val parquetRequestedSchema = if (parquetMrCompatibility) {
    +      // Parquet-mr will throw an exception if we try to read a superset of the file's schema.
    +      // Therefore, we intersect our clipped schema with the underlying file's schema
    +      ParquetReadSupport.intersectParquetGroups(clippedParquetSchema, context.getFileSchema)
    +        .map(intersectionGroup =>
    +          new MessageType(intersectionGroup.getName, intersectionGroup.getFields))
    +        .getOrElse(ParquetSchemaConverter.EMPTY_MESSAGE)
    +    } else {
    +      // Spark's built-in Parquet reader will throw an exception in some cases if the requested
    +      // schema is not the same as the clipped schema
    --- End diff --
    
    cc @rdblue @mswit-databricks Do you know the root cause? 


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    retest this please


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r189491063
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ColumnarFileFormat.scala ---
    @@ -0,0 +1,32 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources
    +
    +import org.apache.spark.sql.SparkSession
    +import org.apache.spark.sql.types.StructType
    +
    +/**
    + * An optional mix-in for columnar [[FileFormat]]s. This trait provides some helpful metadata when
    + * debugging a physical query plan.
    + */
    +private[sql] trait ColumnarFileFormat {
    --- End diff --
    
    Can we do this in a separate PR?  No need to block this PR due to the discussion about this implementation. 


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r190485768
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala ---
    @@ -286,7 +286,19 @@ case class FileSourceScanExec(
           } getOrElse {
             metadata
           }
    -    withOptPartitionCount
    +    val withOptColumnCount = relation.fileFormat match {
    +      case columnar: ColumnarFileFormat =>
    +        SparkSession
    +          .getActiveSession
    +          .map { sparkSession =>
    +            val columnCount = columnar.columnCountForSchema(sparkSession, requiredSchema)
    +            withOptPartitionCount + ("ColumnCount" -> columnCount.toString)
    --- End diff --
    
    Replied above.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/4204/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman Sorry for the delay. Super busy during the Spark summit. Will continue the code review in the next few days. 


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #91442 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91442/testReport)** for PR 21320 at commit [`a8e080f`](https://github.com/apache/spark/commit/a8e080fc8853e67367a60078d18758c0c4395c9b).


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Hi @jainaks. I can see why your query would not work. In the example you provide, if you refer to the column as `name.First`, does your query succeed?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @gatorsmile I believe this is the PR you requested for review.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94987/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution/3792/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2017/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1817/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    **[Test build #94476 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94476/testReport)** for PR 21320 at commit [`3a833db`](https://github.com/apache/spark/commit/3a833db898d6068c8eda11a635d9053a5bb471d7).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    retest this please



---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2391/
    Test PASSed.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r199538123
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala ---
    @@ -0,0 +1,153 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.datasources.parquet
    +
    +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, NamedExpression}
    +import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, ProjectionOverSchema, SelectedField}
    +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
    +import org.apache.spark.sql.catalyst.rules.Rule
    +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType}
    +
    +/**
    + * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a
    + * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the
    + * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a
    + * SQL column, and a nested Parquet column corresponds to a [[StructField]].
    + */
    +private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] {
    +  override def apply(plan: LogicalPlan): LogicalPlan =
    +    if (SQLConf.get.nestedSchemaPruningEnabled) {
    +      apply0(plan)
    +    } else {
    +      plan
    +    }
    +
    +  private def apply0(plan: LogicalPlan): LogicalPlan =
    +    plan transformDown {
    +      case op @ PhysicalOperation(projects, filters,
    +          l @ LogicalRelation(hadoopFsRelation @ HadoopFsRelation(_, _,
    +            dataSchema, _, parquetFormat: ParquetFileFormat, _), _, _, _)) =>
    +        val projectionRootFields = projects.flatMap(getRootFields)
    +        val filterRootFields = filters.flatMap(getRootFields)
    +        val requestedRootFields = (projectionRootFields ++ filterRootFields).distinct
    +
    +        // If [[requestedRootFields]] includes a nested field, continue. Otherwise,
    +        // return [[op]]
    +        if (requestedRootFields.exists { case RootField(_, derivedFromAtt) => !derivedFromAtt }) {
    +          val prunedSchema = requestedRootFields
    +            .map { case RootField(field, _) => StructType(Array(field)) }
    +            .reduceLeft(_ merge _)
    +          val dataSchemaFieldNames = dataSchema.fieldNames.toSet
    --- End diff --
    
    dataSchema may also contains partition columns(see the doc of `HadoopFsRelation`), is this rule prepared for this case?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > @mallman if you're planning on making more code changes, would you be willing to work on a shared branch or something? I've been working to incorporate the CR comments.
    
    No, however if you want to open a PR against the VideoAmp spark-4502-parquet_column_pruning-foundation branch I will review your changes.


---

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


[GitHub] spark pull request #21320: [SPARK-4502][SQL] Parquet nested column pruning -...

Posted by ajacques <gi...@git.apache.org>.
Github user ajacques commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21320#discussion_r205329633
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/ProjectionOverSchema.scala ---
    @@ -0,0 +1,62 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.planning
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * A Scala extractor that projects an expression over a given schema. Data types,
    + * field indexes and field counts of complex type extractors and attributes
    + * are adjusted to fit the schema. All other expressions are left as-is. This
    + * class is motivated by columnar nested schema pruning.
    + */
    +case class ProjectionOverSchema(schema: StructType) {
    --- End diff --
    
    We can move this to `sql.execution` if we move all three classes: `ProjectionOverSchema`, `GetStructFieldObject`, and `SelectedField`. Is there a difference in the catalyst.planning vs the execution packages?


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by mallman <gi...@git.apache.org>.
Github user mallman commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    > Could we move the changes made in ParquetReadSupport.scala to a separate PR? Then, we can merge this PR very quickly.
    
    If I remove the changes to `ParquetReadSupport.scala`, then four tests fail in `ParquetSchemaPruningSuite.scala`.
    
    I don't think we should/can proceed without addressing the issue of reading from two parquet files with identical column names and types but different ordering of those columns in their respective file schema. Personally, I think the fact that the Spark parquet reader appears to assume the same column order in otherwise compatible schema across files is a bug. I think column selection should be by name, not index. The parquet-mr reader behaves that way.
    
    As a stop-gap alternative, I suppose we could disable the built-in reader if parquet schema pruning is turned on. But I think that would be a rather ugly, invasive and confusing hack.
    
    Of course I'm open to other ideas as well.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/1209/
    Test PASSed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #21320: [SPARK-4502][SQL] Parquet nested column pruning - founda...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21320
  
    @mallman Really appreciate for your selfless contributions to Apache Spark! In Spark 2.1, you already made a critical contribution for scalable partition handling. In the upcoming release, the Parquet nested column pruning is one of the major features we plan to announce in Spark SQL. I believe the whole community really appreciates your selfless contributions. 
    
    Sorry for the delay. Recently, I am still doing the review in my local repository, although I did not post the comment. You know, when the PR is huge, it might take more time to finish the review. No matter who will continue the work, the first auhtor of this PR will be assigned to you for sure. PLEASE do not feel frustrated. Also thank you for your patience to address the comments again and again and again. The community needs you! 


---

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