Skip to content

feat: Optimize ORDER BY by Pruning Functionally Redundant Sort Keys#21362

Open
xiedeyantu wants to merge 9 commits intoapache:mainfrom
xiedeyantu:sortkey
Open

feat: Optimize ORDER BY by Pruning Functionally Redundant Sort Keys#21362
xiedeyantu wants to merge 9 commits intoapache:mainfrom
xiedeyantu:sortkey

Conversation

@xiedeyantu
Copy link
Copy Markdown
Member

Which issue does this PR close?

Rationale for this change

This PR adds functional-dependency-based simplification for ORDER BY clauses. When an earlier sort key already functionally determines a later key, the later key is redundant and can be removed without changing query semantics. This reduces unnecessary sorting work and avoids carrying extra sort keys through planning and execution.

What changes are included in this PR?

This PR extends the existing functional dependency utilities with a helper for pruning redundant sort keys, and wires that helper into eliminate_duplicated_expr so Sort nodes can be simplified during optimization. It also adds regression coverage for both the positive case, where a trailing sort key is removed, and the negative case, where sort order prevents pruning.

Are these changes tested?

Yes. I added unit tests covering:

  • removal of a functionally redundant trailing ORDER BY key
  • preservation of ordering when the dependent column appears before its determinant

I also ran cargo test -p datafusion-optimizer eliminate_duplicated_expr -- --nocapture successfully, and cargo fmt --all passes.

Are there any user-facing changes?

Yes, but only in query planning behavior. Some queries with redundant ORDER BY keys may produce simpler plans and run more efficiently. There are no public API changes.

@github-actions github-actions bot added optimizer Optimizer rules common Related to common crate labels Apr 4, 2026
@github-actions github-actions bot added the sqllogictest SQL Logic Tests (.slt) label Apr 4, 2026
Copy link
Copy Markdown
Contributor

@neilconway neilconway left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good overall! A few minor suggestions.

@xiedeyantu
Copy link
Copy Markdown
Member Author

Looks good overall! A few minor suggestions.

@neilconway Thank you for your comments and suggestions; I think the scenario you mentioned—changing order by deptno, total_sal, abs(deptno) to order by deptno, abs(deptno)—is an excellent point. We should definitely support this; our previous restrictions were indeed a bit too strict. We could even go a step further in the future by leveraging injective functions for additional optimizations (though it might be better to implement this in a separate PR). Regarding the rest of your comments, I have attempted to address each one by submitting a corresponding commit. Could you please take another look and review them? Thank you once again for your thorough reviews of every single one of my PRs!

Copy link
Copy Markdown
Contributor

@neilconway neilconway left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for iterating on this!

Can you "resolve" comment threads for review comments you believe have been addressed, please?

@xiedeyantu
Copy link
Copy Markdown
Member Author

Thanks for iterating on this!

Can you "resolve" comment threads for review comments you believe have been addressed, please?

@neilconway My previous understanding was that the issue would only be marked as "resolved" after I had fixed it and the reviewer had confirmed that everything was in order; that is why I didn't click the button. Going forward, I will follow your suggestion. Thank you!

@neilconway
Copy link
Copy Markdown
Contributor

My previous understanding was that the issue would only be marked as "resolved" after I had fixed it and the reviewer had confirmed that everything was in order; that is why I didn't click the button.

I think it's simpler if the PR submitter just proactively "resolves" comments they believe have been resolved; if the reviewer disagrees, they can always reopen the comment thread.

Copy link
Copy Markdown
Contributor

@neilconway neilconway left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good to me! Nice work.

@alamb PR looks reasonable to me.

@xiedeyantu
Copy link
Copy Markdown
Member Author

My previous understanding was that the issue would only be marked as "resolved" after I had fixed it and the reviewer had confirmed that everything was in order; that is why I didn't click the button.

I think it's simpler if the PR submitter just proactively "resolves" comments they believe have been resolved; if the reviewer disagrees, they can always reopen the comment thread.

I completely agree; this will make it much easier for the reviewer to conduct the next round of reviews. Thank you!

Copy link
Copy Markdown
Contributor

@alamb alamb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure about the logic in this one. Thanks @neilconway and @xiedeyantu

----
logical_plan
01)Sort: table_with_ordered_pk.c1 ASC NULLS LAST, table_with_ordered_pk.c2 ASC NULLS LAST
01)Sort: table_with_ordered_pk.c1 ASC NULLS LAST
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't understand this change -- the query requires ORDR BY c1, c2 but now the query only sorts on c1. The primary key on c1 means there are no duplicates, but how does that ensure it is also ordered by c2

For example what about

INSERT INTO table VALUES (1,2)
INSERT INTO table VALUES (2,1)

That is still ordered by c1, but if you don't also sort of c2, you'll end up with the wrong sort

Copy link
Copy Markdown
Contributor

@neilconway neilconway Apr 10, 2026

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, the optimization seems sound to me. If c1 functionally determines c2, we know that each distinct c1 value is associated with exactly one c2 value. So sorting by c1 is sufficient; adding in c2 as a tiebreaker / secondary sort key is never useful.

In the example, if c1 is a PK of table, sorting by c1 is sufficient to get the right sort order -- there will never be two rows with the same c1 value.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@alamb I fully agree with Neil's explanation: functional dependency serves precisely to guarantee this property—for instance, an injective function inherently exhibits the characteristics of functional dependency. In the context of ORDER BY, an additional aspect of ordering comes into play, requiring evaluation based on the sequence of the original fields (as explained in the accompanying code); this ensures that the semantics remain correct after the ORDER BY fields have been eliminated.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

common Related to common crate optimizer Optimizer rules sqllogictest SQL Logic Tests (.slt)

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Optimize ORDER BY by Pruning Functionally Redundant Sort Keys

3 participants