Skip to content

feat: TVF Part 7/X Final PR of adaptation changes#26445

Merged
mohsaka merged 11 commits intoprestodb:masterfrom
mohsaka:tvf_analyzer_final
Jan 2, 2026
Merged

feat: TVF Part 7/X Final PR of adaptation changes#26445
mohsaka merged 11 commits intoprestodb:masterfrom
mohsaka:tvf_analyzer_final

Conversation

@mohsaka
Copy link
Copy Markdown
Contributor

@mohsaka mohsaka commented Oct 27, 2025

Description

This PR contains all final changes to TVF functionality.

Motivation and Context

Completes the addition of TVF support in Presto.

Impact

Test Plan

Added new test cases.

Rule test cases:
TestTransformTableFunctionToTableFunctionProcessor
TestPruneTableFunctionProcessorColumns
TestPruneTableFunctionProcessorSourceColumns
TestRemoveRedundantTableFunction
TestRewriteExcludeColumnsFunctionToProjection

Planner test case:
planner/TestTableFunctionInvocation

System TVF Test Cases:
TestExcludeColumnsFunction
TestSequenceFunction

Re-ran previous test cases to check for regressions:
test/TestTableFunctionInvocation
TestTableFunctionRegistry
TestAnalyzer

Contributor checklist

  • Please make sure your submission complies with our contributing guide, in particular code style and commit standards.
  • PR description addresses the issue accurately and concisely. If the change is non-trivial, a GitHub Issue is referenced.
  • Documented new properties (with its default value), SQL syntax, functions, or other functionality.
  • If release notes are required, they follow the release notes guidelines.
  • Adequate tests were added if applicable.
  • CI passed.
  • If adding new dependencies, verified they have an OpenSSF Scorecard score of 5.0 or higher (or obtained explicit TSC approval for lower scores).

Release Notes

Please follow release notes guidelines and fill in the release notes below.

== NO RELEASE NOTE ==

@prestodb-ci prestodb-ci added the from:IBM PR from IBM label Oct 27, 2025
@mohsaka mohsaka changed the title Final analyzer/planner/optimizer changes for tvf feat: Final analyzer/planner/optimizer changes for tvf Oct 27, 2025
Copy link
Copy Markdown
Contributor

@sourcery-ai sourcery-ai bot left a comment

Choose a reason for hiding this comment

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

Sorry @mohsaka, your pull request is larger than the review limit of 150000 diff characters

@mohsaka mohsaka force-pushed the tvf_analyzer_final branch from 056ebac to 4f63c71 Compare October 27, 2025 23:06
@mohsaka mohsaka changed the title feat: Final analyzer/planner/optimizer changes for tvf feat: TVF Part 7/X Final analyzer/planner/optimizer changes for tvf Oct 27, 2025
@mohsaka mohsaka requested a review from aditi-pandit October 27, 2025 23:27
@mohsaka mohsaka marked this pull request as ready for review October 27, 2025 23:27
@prestodb-ci prestodb-ci requested review from a team and nmahadevuni and removed request for a team October 27, 2025 23:27
Copy link
Copy Markdown
Contributor

@sourcery-ai sourcery-ai bot left a comment

Choose a reason for hiding this comment

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

Sorry @mohsaka, your pull request is larger than the review limit of 150000 diff characters

Copy link
Copy Markdown
Contributor

@aditi-pandit aditi-pandit left a comment

Choose a reason for hiding this comment

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

Thanks @mohsaka.

It might get easier to review the code if we split as follows :
i) Changes in TestingTableFunctions.java
ii) Add the TableFunctionNode changes and the basic structural wiring (in Printer, GraphVizPrinter, PlanBuilder, PlanMatcher)
iii) Similar changes ffor TableFunctionProcessorNode..The PlanNode definition and the basic structural wiring like above.
iv) There are bunch of changes like those in Field.java, StatementAnalyzer changes which can be one off changes and can be made independent of this PR.
v) Basic Planner rule for ImplementTableFunctionSource
vi) PruneTableFunction* rules + TestPrune*
vii) Remove* rules + TestRemove*

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.

Move this change to a separate PR.

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.

Don't understand the reason for the changes in this file. Why did you have to make this class public ?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Coerce function needed for RelationPlanner

                    QueryPlanner.PlanAndMappings copartitionCoercions = partitionQueryPlanner.coerce(sourcePlanBuilder, partitioningColumns, analysis, idAllocator, variableAllocator, metadata);
                    sourcePlanBuilder = copartitionCoercions.getSubPlan();
                    partitionBy = partitioningColumns.stream()
                            .map(copartitionCoercions::get)
                            .collect(toImmutableList());

Scope inputScope = analysis.getScope(tableArgumentsByName.get(name).getRelation());
columns.stream()
.filter(column -> column < 0 || column >= inputScope.getRelationType().getAllFieldCount()) // hidden columns can be required as well as visible columns
.filter(column -> column < 0 || column >= inputScope.getRelationType().getVisibleFieldCount())
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.

This can be added as a single change without the others.

Copy link
Copy Markdown
Contributor

@aditi-pandit aditi-pandit left a comment

Choose a reason for hiding this comment

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

This code looks good. Will do one more read.

// for processing or for pass-through. null value in the marker column indicates that the value at the same
// position in the source column should not be processed or passed-through.
// the mapping is only present if there are two or more sources.
private final Optional<Map<VariableReferenceExpression, VariableReferenceExpression>> markerVariables;
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.

Can you add a comment with an example for this.

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.

@aditi-pandit Thanks. I've added an example in the comment.

return orderBy;
}

static void addPassthroughColumns(ImmutableList.Builder<VariableReferenceExpression> outputVariables,
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.

Can you add a comment about the usage of this function and explanation for each parameter

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.

@aditi-pandit Thanks. I've added a comment for the explannation.

@aditi-pandit
Copy link
Copy Markdown
Contributor

@jaystarshot : PTAL. This code has the main planner changes for Table function.

@xin-zhang2 xin-zhang2 force-pushed the tvf_analyzer_final branch 4 times, most recently from a952a34 to 33da439 Compare November 11, 2025 10:57
* - source T2(a2, b2)
* </pre>
*/
public class ImplementTableFunctionSource
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

the naming of this rule is non intuitive, can it be improved? eg
TransformTableFunctionToProcessorNodeRule

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.

@jaystarshot Thanks. I've renamed it to TransformTableFunctionToTableFunctionProcessor. What do you think of this name?

Copy link
Copy Markdown
Member

@jaystarshot jaystarshot left a comment

Choose a reason for hiding this comment

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

I think this PR is missing end to end integration (execution) tests, without those its hard to say if the addExchanges etc is correct. Can you please add them

@tdcmeehan tdcmeehan self-assigned this Dec 10, 2025
Copy link
Copy Markdown
Member

@jaystarshot jaystarshot left a comment

Choose a reason for hiding this comment

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

I have reviewed most of the planner changes only "SymbolMapper.java" is left which i will review.

return Optional.empty();
});

return translatedProperties.unordered(true);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: maybe a comment here explaining why conservative in being unordered

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added comment. Main reason is that the user can pretty much do anything with the rows they are provided. So we don't have a guarantee that they are ordered after table function application.

PlanNode newSource = node.getSources().get(i).accept(this, context);
newSources.add(newSource);

SymbolMapper inputMapper = new SymbolMapper(new HashMap<>(), warningCollector);
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This input mapper is empty, this doesn't look correct

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This was pretty wrong. Fixed a few things.

  1. Mapper should be used from Rewriter mapping.
  2. We should have been using context.rewrite instead of accept. Following the convention of the other Nodes.

Thanks!

@mohsaka
Copy link
Copy Markdown
Contributor Author

mohsaka commented Dec 12, 2025

I have reviewed most of the planner changes only "SymbolMapper.java" is left which i will review.

Thank you for the review! I will take a look today or monday.

@tdcmeehan tdcmeehan self-requested a review December 12, 2025 19:14
final class Processed
implements TableFunctionProcessorState
{
private final boolean usedInput;
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.

Can you document what usedInput signifies so folks can understand the semantics?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added comment

* @param input a tuple of {@link Page} including one page for each table function's input table.
* Pages list is ordered according to the corresponding argument specifications in {@link ConnectorTableFunction}.
* A page for an argument consists of columns requested during analysis (see {@link TableFunctionAnalysis#getRequiredColumns()}}.
* If any of the sources is fully processed, {@code Optional.empty)()} is returned for that source.
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.

Suggested change
* If any of the sources is fully processed, {@code Optional.empty)()} is returned for that source.
* If any of the sources is fully processed, {@code Optional.empty()} is returned for that source.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Good find.

/**
* This method processes a split. It is called multiple times until the whole output for the split is produced.
*
* @param split a {@link ConnectorSplit} representing a subtask.
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.

Let's document that this is Nullable and when it's expected to be null.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added description for when table function is labeled KEEP WITH EMPTY and has no input.

public interface TableFunctionSplitProcessor
{
/**
* This method processes a split. It is called multiple times until the whole output for the split is produced.
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.

Should we make it clear this is 1:1 with a split?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added comment explaining that the Split processor is one to one with a split.

import static com.google.common.base.Preconditions.checkArgument;
import static java.lang.String.format;

public class Sequence
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.

Let's add documentation for this. This probably entails a new section for TVFs, next to our existing functions.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Agreed, we plan on doing something similar to
https://trino.io/docs/current/functions/table.html

import static java.util.Locale.ENGLISH;
import static java.util.stream.Collectors.joining;

public class ExcludeColumns
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.

We should add documentation for this as well.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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


private final OperatorContext operatorContext;

private final PageBuffer pageBuffer = new PageBuffer();
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.

My IDE is showing this is unused.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I'm not too sure on why this was changed from the old implementation. But switched back to the old one which was simpler.

Comment on lines +152 to +153
// Fallback if needed
return getFunctionId(split, tableFunctionSplitResolvers);
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.

Can you explain why we need this Exception-based fallback? Is there any way to refactor to avoid the need to do this? It's far preferable to just let Exceptions bubble up.

.map(RowType.Field::getName)
.filter(Optional::isPresent)
.map(Optional::get)
.map(name -> name.toLowerCase(ENGLISH))
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 think this will break any connector that has case sensitive identifiers.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Agreed. There's a comment left by kasiafi acknowledging this issue and a TODO above.

            // column names in DescriptorArgument are canonical wrt SQL identifier semantics.
            // column names in TableArgument are not canonical wrt SQL identifier semantics, as they are taken from the corresponding RelationType.
            // because of that, we match the excluded columns names case-insensitive
            // TODO: apply proper identifier semantics

import static com.google.common.base.Preconditions.checkState;
import static java.util.Objects.requireNonNull;

public class PageBuffer
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.

This looks completely unused.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Used now. Not sure why it was not used before. I'm guessing I missed something when bringing code in by component.

@mohsaka
Copy link
Copy Markdown
Contributor Author

mohsaka commented Dec 13, 2025

@jaystarshot @tdcmeehan Thank you for the thorough reviews. Really appreciate the time you take to do them. I have addressed the comments so please take a second look when you have the chance. Thanks again!

mohsaka and others added 10 commits December 13, 2025 10:56
…nPlanner and ExcludeColumns optimizer rule.

Co-authored-by: kasiafi <30203062+kasiafi@users.noreply.github.com>
Co-authored-by: Xin Zhang <desertsxin@gmail.com>
Co-authored-by: kasiafi <30203062+kasiafi@users.noreply.github.com>
Co-authored-by: mohsaka <135669458+mohsaka@users.noreply.github.com>
Co-authored-by: kasiafi <30203062+kasiafi@users.noreply.github.com>
Co-authored-by: mohsaka <135669458+mohsaka@users.noreply.github.com>
Changes adapted from trino/PR#16584
Author: kasiafi

Co-authored-by: kasiafi <30203062+kasiafi@users.noreply.github.com>
Co-authored-by: Xin Zhang <desertsxin@gmail.com>
Changes adapted from trino/PR#16716
Author: kasiafi

Co-authored-by: kasiafi <30203062+kasiafi@users.noreply.github.com>
Changes adapted from trino/PR#25493
Author: kasiafi

Co-authored-by: kasiafi <30203062+kasiafi@users.noreply.github.com>
// as if it was a single partition. Alternatively, it could be split into smaller partitions of arbitrary size.
DataOrganizationSpecification specification = argumentProperties.getSpecification().orElse(UNORDERED_SINGLE_PARTITION);

PlanNode innerWindow = new WindowNode(
Copy link
Copy Markdown
Contributor

@aditi-pandit aditi-pandit Dec 18, 2025

Choose a reason for hiding this comment

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

@mohsaka : We don't need to do window over window as a window node can have 2 window functions that have the same partition by and order by. Can you try combining the 2 windows with a single function list and see what happens ?

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.

@mohsaka : I'm fine with doing this as a follow up PR.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

@aditi-pandit We originally had it as a single window function, which can be viewed in one of our really old PR's
https://github.com/mohsaka/presto/blob/c2d64577387f41bd0f1270c55b0fa851920eb6bb/presto-main/src/main/java/com/facebook/presto/sql/planner/iterative/rule/ImplementTableFunctionSource.java#L358

However it caused us to modify the WindowFilterPushDown rule and remove this check

checkState(node.getWindowFunctions().size() == 1, "WindowFilterPushdown requires that WindowNodes contain exactly one window function");

We didn't think this was proper, so we had to split it in 2.

Copy link
Copy Markdown
Contributor

@tdcmeehan tdcmeehan left a comment

Choose a reason for hiding this comment

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

Execution side LGTM

Copy link
Copy Markdown
Member

@jaystarshot jaystarshot left a comment

Choose a reason for hiding this comment

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

Planning changes LGTM

@mohsaka mohsaka changed the title feat: TVF Part 7/X Final analyzer/planner/optimizer changes for tvf feat: TVF Part 7/X Final PR of adaptation changes Jan 2, 2026
@mohsaka mohsaka merged commit c2b0f5f into prestodb:master Jan 2, 2026
85 of 93 checks passed
@prestodb-ci
Copy link
Copy Markdown
Contributor

@mohsaka imported this issue as lakehouse/presto #26445

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

Labels

from:IBM PR from IBM

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants