Skip to content

[WIP] Fix problem in NestedColumnAliasing.scala , replaceWithAliases in connection with Generate plan node #49061

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 4 commits into from

Conversation

trohwer
Copy link

@trohwer trohwer commented Dec 4, 2024

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.

Copy link

We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!

@github-actions github-actions bot added the Stale label Mar 16, 2025
@github-actions github-actions bot closed this Mar 17, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

1 participant