trohwer opened a new pull request, #49061:
URL: https://github.com/apache/spark/pull/49061

   When one uses replaceWithChildren, one has to be careful with Generate plan 
nodes. Generate contains a list unrequiredChildIndex of unneeded child outputs 
in the Generate output. This data has to be adjusted accordingly. Otherwise an 
incorrect plan may be generated during optimisation. Here is an example (tested 
with Spark 3.5.3):
   
   from pyspark.sql import SparkSession
   
   session= SparkSession.builder.master("local").getOrCreate()
   
   session.sql("""
   select
       named_struct(
             'b', '',
             'c', '',
             'd', array(named_struct('f', '', 'g', '')),
             'e', ''
       ) as a
   """).write.mode("overwrite").parquet("tmp")
   
   df= session.read.parquet("tmp")
   df.createOrReplaceTempView("tmp")
   
   sql="""
   SELECT
   a.b f1, a.c f2, x.f,
   STACK(1, y) as (z)
   FROM tmp
   LATERAL VIEW POSEXPLODE_OUTER(a.d) as y, x
   """
   
   session.sql(sql).explain()
   
   #== Physical Plan ==                                                         
    
   #*(1) !Project [_extract_b#21 AS f1#5, _extract_c#19 AS f2#6, _extract_f#20 
AS f#12, z#13]
   #+- *(1) Generate stack(1, y#8), [_extract_b#21, _extract_f#20], false, 
[z#13]
   #   +- *(1) Project [_extract_b#21, y#8, x#9 AS _extract_f#20]
   #      +- *(1) Generate posexplode(_extract_f#26), [_extract_b#21], true, 
[y#8, x#9]
   #         +- *(1) Project [a#3.b AS _extract_b#21, a#3.d.f AS _extract_f#26]
   #            +- *(1) ColumnarToRow
   #               +- FileScan parquet [a#3] Batched: true, DataFilters: [], 
Format: Parquet, Location: InMemoryFileIndex(1 
paths)[file:/home/pa/test/spark-bug/tmp], PartitionFilters: [], PushedFilters: 
[], ReadSchema: struct<a:struct<b:string,d:array<struct<f:string>>>>
   
   session.sql(sql).show()
   
   # java.lang.IllegalStateException: Couldn't find _extract_c#54 in 
[_extract_b#56,_extract_f#55,z#36]
   
   
   
   One can see, that the generated plan is invalid (_extract_c_#19 is missing 
in the in previous Project) and yields an error during execution. With this 
fix, the problem does not occur.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to