You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
feat: remove __unnest_placeholder from struct unnest projection (#21725)
## Which issue does this PR close?
<!--
We generally require a GitHub issue to be filed for all bug fixes and
enhancements and this helps us generate change logs for our releases.
You can link an issue to this PR using the GitHub syntax. For example
`Closes#123` indicates that this PR will close issue #123.
-->
- Closes#16894
## What changes are included in this PR?
- User-visible struct-UNNEST columns no longer expose
`__unnest_placeholder`.
- Internal Unnest planning still uses placeholders.
- This adds a final Projection layer in explain plans wherever struct
UNNEST output is published.
<!--
There is no need to duplicate the description in the issue here but it
is sometimes worth providing a summary of the individual changes in this
PR.
-->
## Are these changes tested?
Yes.
<!--
We typically require tests for all PRs in order to:
1. Prevent the code from being accidentally broken by subsequent changes
2. Serve as another way to document the expected behavior of the code
If tests are not included in your PR, please explain why (for example,
are they covered by existing tests)?
-->
## Are there any user-facing changes?
Yes. The aliases of unnested columns will be changed.
<!--
If there are user-facing changes then we may require documentation to be
updated before approving the PR.
-->
<!--
If there are any breaking changes to public APIs, please add the `api
change` label.
-->
---------
Co-authored-by: Andrew Lamb <andrew@nerdnetworks.org>
Projection: __unnest_placeholder(unnest_table.struct_col).field1, __unnest_placeholder(unnest_table.struct_col).field2, __unnest_placeholder(unnest_table.array_col,depth=1) AS UNNEST(unnest_table.array_col), unnest_table.struct_col, unnest_table.array_col
991
+
Projection: __unnest_placeholder(unnest_table.struct_col).field1 AS unnest_table.struct_col.field1, __unnest_placeholder(unnest_table.struct_col).field2 AS unnest_table.struct_col.field2, __unnest_placeholder(unnest_table.array_col,depth=1) AS UNNEST(unnest_table.array_col), unnest_table.struct_col, unnest_table.array_col
Projection: unnest_table.struct_col AS __unnest_placeholder(unnest_table.struct_col), unnest_table.array_col AS __unnest_placeholder(unnest_table.array_col), unnest_table.struct_col, unnest_table.array_col
02)--Projection: __unnest_placeholder(recursive_unnest_table.column3,depth=1) AS UNNEST(recursive_unnest_table.column3) AS __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), recursive_unnest_table.column3
01)Projection: __unnest_placeholder(UNNEST(recursive_unnest_table.column3)).c0 AS UNNEST(recursive_unnest_table.column3).c0, __unnest_placeholder(UNNEST(recursive_unnest_table.column3)).c1 AS UNNEST(recursive_unnest_table.column3).c1, recursive_unnest_table.column3
03)----Projection: __unnest_placeholder(recursive_unnest_table.column3,depth=1) AS UNNEST(recursive_unnest_table.column3) AS __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), recursive_unnest_table.column3
03)----ProjectionExec: expr=[__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3]
620
-
04)------UnnestExec
621
-
05)--------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3]
01)ProjectionExec: expr=[__unnest_placeholder(UNNEST(recursive_unnest_table.column3)).c0@0 as UNNEST(recursive_unnest_table.column3).c0, __unnest_placeholder(UNNEST(recursive_unnest_table.column3)).c1@1 as UNNEST(recursive_unnest_table.column3).c1, column3@2 as column3]
04)------ProjectionExec: expr=[__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3]
622
+
05)--------UnnestExec
623
+
06)----------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3]
04)------Projection: struct(t.column1, t.column2, t.column3) AS __unnest_placeholder(struct(t.column1,t.column2,t.column3))
1105
-
05)--------TableScan: t projection=[column1, column2, column3]
1105
+
03)----Projection: __unnest_placeholder(struct(t.column1,t.column2,t.column3)).c0 AS struct(t.column1,t.column2,t.column3).c0, __unnest_placeholder(struct(t.column1,t.column2,t.column3)).c1 AS struct(t.column1,t.column2,t.column3).c1, __unnest_placeholder(struct(t.column1,t.column2,t.column3)).c2 AS struct(t.column1,t.column2,t.column3).c2
02)--ProjectionExec: expr=[__unnest_placeholder(struct(t.column1,t.column2,t.column3)).c0@0 as struct(t.column1,t.column2,t.column3).c0, __unnest_placeholder(struct(t.column1,t.column2,t.column3)).c1@1 as struct(t.column1,t.column2,t.column3).c1, __unnest_placeholder(struct(t.column1,t.column2,t.column3)).c2@2 as struct(t.column1,t.column2,t.column3).c2]
1112
+
03)----UnnestExec
1113
+
04)------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/unnest/ordered_tuples.parquet]]}, projection=[struct(column1@0, column2@1, column3@2) as __unnest_placeholder(struct(t.column1,t.column2,t.column3))], file_type=parquet
1110
1114
1111
1115
# cleanup
1112
1116
statement ok
@@ -1152,12 +1156,14 @@ EXPLAIN SELECT UNNEST(column1), column2 FROM t ORDER BY column2;
03)----Projection: t.column1 AS __unnest_placeholder(t.column1), t.column2
1157
-
04)------TableScan: t projection=[column1, column2]
1159
+
02)--Projection: __unnest_placeholder(t.column1).s1 AS t.column1.s1, __unnest_placeholder(t.column1).s2 AS t.column1.s2, __unnest_placeholder(t.column1).s3 AS t.column1.s3, t.column2
01)ProjectionExec: expr=[__unnest_placeholder(t.column1).s1@0 as t.column1.s1, __unnest_placeholder(t.column1).s2@1 as t.column1.s2, __unnest_placeholder(t.column1).s3@2 as t.column1.s3, column2@3 as column2]
02)--Projection: __unnest_placeholder(t.column1,depth=2) AS UNNEST(UNNEST(t.column1)), __unnest_placeholder(t.column2,depth=1) AS UNNEST(t.column2), __unnest_placeholder(t.column3).s1, __unnest_placeholder(t.column3).s2, __unnest_placeholder(t.column3).s3, t.column4
1229
+
02)--Projection: __unnest_placeholder(t.column1,depth=2) AS UNNEST(UNNEST(t.column1)), __unnest_placeholder(t.column2,depth=1) AS UNNEST(t.column2), __unnest_placeholder(t.column3).s1 AS t.column3.s1, __unnest_placeholder(t.column3).s2 AS t.column3.s2, __unnest_placeholder(t.column3).s3 AS t.column3.s3, t.column4
04)------Projection: t.column1 AS __unnest_placeholder(t.column1), t.column2 AS __unnest_placeholder(t.column2), t.column3 AS __unnest_placeholder(t.column3), t.column4
1226
1232
05)--------TableScan: t projection=[column1, column2, column3, column4]
1227
1233
physical_plan
1228
-
01)ProjectionExec: expr=[__unnest_placeholder(t.column1,depth=2)@0 as UNNEST(UNNEST(t.column1)), __unnest_placeholder(t.column2,depth=1)@1 as UNNEST(t.column2), __unnest_placeholder(t.column3).s1@2 as __unnest_placeholder(t.column3).s1, __unnest_placeholder(t.column3).s2@3 as __unnest_placeholder(t.column3).s2, __unnest_placeholder(t.column3).s3@4 as __unnest_placeholder(t.column3).s3, column4@5 as column4]
1234
+
01)ProjectionExec: expr=[__unnest_placeholder(t.column1,depth=2)@0 as UNNEST(UNNEST(t.column1)), __unnest_placeholder(t.column2,depth=1)@1 as UNNEST(t.column2), __unnest_placeholder(t.column3).s1@2 as t.column3.s1, __unnest_placeholder(t.column3).s2@3 as t.column3.s2, __unnest_placeholder(t.column3).s3@4 as t.column3.s3, column4@5 as column4]
1229
1235
02)--UnnestExec
1230
1236
03)----DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/unnest/ordered_struct_arrays.parquet]]}, projection=[column1@0 as __unnest_placeholder(t.column1), column2@1 as __unnest_placeholder(t.column2), column3@2 as __unnest_placeholder(t.column3), column4], output_ordering=[column4@3 ASC NULLS LAST], file_type=parquet
0 commit comments