[ 
https://issues.apache.org/jira/browse/SPARK-29721?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Kai Kang updated SPARK-29721:
-----------------------------
    Description: 
This is a follow up for SPARK-4502. SPARK-4502 correctly addressed column 
pruning for nested structures. However, when explode() is called on a nested 
field, all columns for that nested structure is still fetched from data source.

We are working on a project to create a parquet store for a big pre-joined 
table between two tables that has one-to-many relationship, and this is a 
blocking issue for us.

 

The following code illustrates the issue. 

Part 1: loading some nested data
{noformat}
val jsonStr = """{
 "items": [
   {"itemId": 1, "itemData": "a"},
   {"itemId": 2, "itemData": "b"}
 ]
}"""
val df = spark.read.json(Seq(jsonStr).toDS)
df.write.format("parquet").mode("overwrite").saveAsTable("persisted")
{noformat}
 
Part 2: reading it back and explaining the queries
val read = spark.table("persisted")
spark.conf.set("spark.sql.optimizer.nestedSchemaPruning.enabled", true)
read.select($"items.itemId").explain(true) // pruned, only loading itemId
read.select(explode($"items.itemId")).explain(true) // not pruned, loading both 
itemId and itemData
 

  was:
This is a follow up for SPARK-4502. SPARK-4502 correctly addressed column 
pruning for nested structures. However, when explode() is called on a nested 
field, all columns for that nested structure is still fetched from data source.

We are working on a project to create a parquet store for a big pre-joined 
table between two tables that has one-to-many relationship, and this is a 
blocking issue for us.

 

The following code illustrates the issue. 

Part 1: loading some nested data
{quote}
val jsonStr = """{
 "items": [
   {"itemId": 1, "itemData": "a"},
   {"itemId": 2, "itemData": "b"}
 ]
}"""
val df = spark.read.json(Seq(jsonStr).toDS)
df.write.format("parquet").mode("overwrite").saveAsTable("persisted")
{quote}
 
Part 2: reading it back and explaining the queries
val read = spark.table("persisted")
spark.conf.set("spark.sql.optimizer.nestedSchemaPruning.enabled", true)
read.select($"items.itemId").explain(true) // pruned, only loading itemId
read.select(explode($"items.itemId")).explain(true) // not pruned, loading both 
itemId and itemData
 


> Spark SQL reads unnecessary nested fields from Parquet after using explode
> --------------------------------------------------------------------------
>
>                 Key: SPARK-29721
>                 URL: https://issues.apache.org/jira/browse/SPARK-29721
>             Project: Spark
>          Issue Type: Improvement
>          Components: SQL
>    Affects Versions: 2.4.4
>            Reporter: Kai Kang
>            Priority: Critical
>
> This is a follow up for SPARK-4502. SPARK-4502 correctly addressed column 
> pruning for nested structures. However, when explode() is called on a nested 
> field, all columns for that nested structure is still fetched from data 
> source.
> We are working on a project to create a parquet store for a big pre-joined 
> table between two tables that has one-to-many relationship, and this is a 
> blocking issue for us.
>  
> The following code illustrates the issue. 
> Part 1: loading some nested data
> {noformat}
> val jsonStr = """{
>  "items": [
>    {"itemId": 1, "itemData": "a"},
>    {"itemId": 2, "itemData": "b"}
>  ]
> }"""
> val df = spark.read.json(Seq(jsonStr).toDS)
> df.write.format("parquet").mode("overwrite").saveAsTable("persisted")
> {noformat}
>  
> Part 2: reading it back and explaining the queries
> val read = spark.table("persisted")
> spark.conf.set("spark.sql.optimizer.nestedSchemaPruning.enabled", true)
> read.select($"items.itemId").explain(true) // pruned, only loading itemId
> read.select(explode($"items.itemId")).explain(true) // not pruned, loading 
> both itemId and itemData
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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

Reply via email to