alamb commented on code in PR #16985:
URL: https://github.com/apache/datafusion/pull/16985#discussion_r2293680506


##########
datafusion/sqllogictest/test_files/unnest.slt:
##########
@@ -941,3 +941,33 @@ where min_height * width1 = (
 )
 ----
 4 7 4 28
+
+## Unnest with ordering on unrelated column is preserved

Review Comment:
   I wrote an additional test, and it doesn't seem to be working as I expect. 
   
   Specifically, I expect this query to not need a sort as the data is already 
sorted on column2:
   
   ```sql
   
   EXPLAIN SELECT UNNEST(column1), column2 FROM t ORDER BY column2;
   ```
   
   However, the physical plan still shows a SortExec:
   ```
   ----
   logical_plan
   01)Sort: t.column2 ASC NULLS LAST
   02)--Projection: __unnest_placeholder(t.column1,depth=1) AS 
UNNEST(t.column1), t.column2
   03)----Unnest: lists[__unnest_placeholder(t.column1)|depth=1] structs[]
   04)------Projection: t.column1 AS __unnest_placeholder(t.column1), t.column2
   05)--------TableScan: t projection=[column1, column2]
   physical_plan
   01)SortExec: expr=[column2@1 ASC NULLS LAST], preserve_partitioning=[false]
   02)--ProjectionExec: expr=[__unnest_placeholder(t.column1,depth=1)@0 as 
UNNEST(t.column1), column2@1 as column2]
   03)----UnnestExec
   04)------ProjectionExec: expr=[column1@0 as __unnest_placeholder(t.column1), 
column2@1 as column2]
   05)--------DataSourceExec: file_groups={1 group: 
[[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/unnest/ordered.parquet]]},
 projection=[column1, column2], output_ordering=[column2@1 ASC NULLS LAST], 
file_type=parquet
   ```
   
   Here is the whole test
   
   ```sql
   
   # Data is ordered by column2 (100, 200, 300, 400)
   statement error DataFusion error: SQL error: ParserError\("Expected: an SQL 
statement, found: COPT at Line: 1, Column: 1"\)
   COPY (
     SELECT * FROM VALUES
       ([1,2,3], 100),
       ([3],     200),
       ([],      300),
       ([3,1],   400)
     ORDER BY column2
    ) TO 'test_files/scratch/unnest/ordered.parquet';
   
   
   statement ok
   CREATE EXTERNAL TABLE t
   STORED AS PARQUET
   LOCATION 'test_files/scratch/unnest/ordered.parquet'
   WITH ORDER (column2)
   
   query ?I
   SELECT * FROM t;
   ----
   [1, 2, 3] 100
   [3] 200
   [] 300
   [3, 1] 400
   
   # data is sorted on column2 already, so no need to sort again
   query II
   SELECT UNNEST(column1), column2 FROM t ORDER BY column2;
   ----
   1 100
   2 100
   3 100
   3 200
   3 400
   1 400
   
   # Explain should not have a SortExec
   query TT
   EXPLAIN SELECT UNNEST(column1), column2 FROM t ORDER BY column2;
   ----
   logical_plan
   01)Sort: t.column2 ASC NULLS LAST
   02)--Projection: __unnest_placeholder(t.column1,depth=1) AS 
UNNEST(t.column1), t.column2
   03)----Unnest: lists[__unnest_placeholder(t.column1)|depth=1] structs[]
   04)------Projection: t.column1 AS __unnest_placeholder(t.column1), t.column2
   05)--------TableScan: t projection=[column1, column2]
   physical_plan
   01)SortExec: expr=[column2@1 ASC NULLS LAST], preserve_partitioning=[false]
   02)--ProjectionExec: expr=[__unnest_placeholder(t.column1,depth=1)@0 as 
UNNEST(t.column1), column2@1 as column2]
   03)----UnnestExec
   04)------ProjectionExec: expr=[column1@0 as __unnest_placeholder(t.column1), 
column2@1 as column2]
   05)--------DataSourceExec: file_groups={1 group: 
[[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/unnest/ordered.parquet]]},
 projection=[column1, column2], output_ordering=[column2@1 ASC NULLS LAST], 
file_type=parquet
   
   # cleanup
   statement ok
   drop table t;
   
   ```



##########
datafusion/sqllogictest/test_files/unnest.slt:
##########
@@ -941,3 +941,33 @@ where min_height * width1 = (
 )
 ----
 4 7 4 28
+
+## Unnest with ordering on unrelated column is preserved

Review Comment:
   I did verify that this test fails without the code change in this PR, but it 
wasn't clear to me why (the new plan has a sort, which the original did not)
   
   ```diff
   diff --git a/datafusion/sqllogictest/test_files/unnest.slt 
b/datafusion/sqllogictest/test_files/unnest.slt
   index 6d5151e1d..eb4abaffc 100644
   --- a/datafusion/sqllogictest/test_files/unnest.slt
   +++ b/datafusion/sqllogictest/test_files/unnest.slt
   @@ -960,14 +960,13 @@ logical_plan
    08)--------------TableScan: range() projection=[value]
    physical_plan
    01)ProjectionExec: expr=[array_agg(unnested.ar)@1 as array_agg(unnested.ar)]
   -02)--AggregateExec: mode=FinalPartitioned, gby=[generated_id@0 as 
generated_id], aggr=[array_agg(unnested.ar)], ordering_mode=Sorted
   -03)----SortExec: expr=[generated_id@0 ASC NULLS LAST], 
preserve_partitioning=[true]
   -04)------CoalesceBatchesExec: target_batch_size=8192
   -05)--------RepartitionExec: partitioning=Hash([generated_id@0], 4), 
input_partitions=4
   -06)----------AggregateExec: mode=Partial, gby=[generated_id@0 as 
generated_id], aggr=[array_agg(unnested.ar)], ordering_mode=Sorted
   -07)------------ProjectionExec: expr=[generated_id@0 as generated_id, 
__unnest_placeholder(make_array(range().value),depth=1)@1 as ar]
   -08)--------------UnnestExec
   -09)----------------ProjectionExec: expr=[row_number() ROWS BETWEEN 
UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as generated_id, 
make_array(value@0) as __unnest_placeholder(make_array(range().value))]
   -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), 
input_partitions=1
   -11)--------------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN 
UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { name: "row_number() ROWS 
BETWEEN UNBOUNDED PRECEDING AND
   UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, 
dict_is_ordered: false, metadata: {} }, frame: ROWS BETWEEN UNBOUNDED PRECEDING 
AND UNBOUNDED FOLLOWING], mode=[Sorte
   d]
   -12)----------------------LazyMemoryExec: partitions=1, 
batch_generators=[range: start=1, end=5, batch_size=8192]
   +02)--AggregateExec: mode=FinalPartitioned, gby=[generated_id@0 as 
generated_id], aggr=[array_agg(unnested.ar)]
   +03)----CoalesceBatchesExec: target_batch_size=8192
   +04)------RepartitionExec: partitioning=Hash([generated_id@0], 4), 
input_partitions=4
   +05)--------AggregateExec: mode=Partial, gby=[generated_id@0 as 
generated_id], aggr=[array_agg(unnested.ar)]
   +06)----------ProjectionExec: expr=[generated_id@0 as generated_id, 
__unnest_placeholder(make_array(range().value),depth=1)@1 as ar]
   +07)------------UnnestExec
   +08)--------------ProjectionExec: expr=[row_number() ROWS BETWEEN UNBOUNDED 
PRECEDING AND UNBOUNDED FOLLOWING@1 as generated_id, make_array(value@0) as 
__unnest_placeholder(make_array(ran
   ge().value))]
   +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), 
input_partitions=1
   +10)------------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN 
UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Field { name: "row_number() ROWS 
BETWEEN UNBOUNDED PRECEDING AND UN
   BOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, 
dict_is_ordered: false, metadata: {} }, frame: ROWS BETWEEN UNBOUNDED PRECEDING 
AND UNBOUNDED FOLLOWING], mode=[Sorted]
   +11)--------------------LazyMemoryExec: partitions=1, 
batch_generators=[range: start=1, end=5, batch_size=8192]
   ```



-- 
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: github-unsubscr...@datafusion.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to