-
Notifications
You must be signed in to change notification settings - Fork 25.2k
[ES|QL] COMPLETION command logical plan optimizer #126763
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
Merged
afoucret
merged 8 commits into
elastic:main
from
afoucret:esql-completion-logical-plan-optimizer
Apr 16, 2025
Merged
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
a93f840
Logical plan optimization for the completion command.
afoucret 9d3200a
Add tests for COMPLETION logical plan optimization.
afoucret adfb565
Ensure filters are pushed down before completion when it is possible.
afoucret 52c6acb
Lint.
afoucret 2161915
Add a comment with an example of filter push down
afoucret 98816dc
[CI] Auto commit changes from spotless
elasticsearchmachine 4eab7d5
Adding some tests for pushable limits optimizer rules.
afoucret 6c4afe4
Merge branch 'main' into esql-completion-logical-plan-optimizer
afoucret File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
18 changes: 18 additions & 0 deletions
18
...rc/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownCompletion.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,18 @@ | ||
/* | ||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one | ||
* or more contributor license agreements. Licensed under the Elastic License | ||
* 2.0; you may not use this file except in compliance with the Elastic License | ||
* 2.0. | ||
*/ | ||
|
||
package org.elasticsearch.xpack.esql.optimizer.rules.logical; | ||
|
||
import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; | ||
import org.elasticsearch.xpack.esql.plan.logical.inference.Completion; | ||
|
||
public final class PushDownCompletion extends OptimizerRules.OptimizerRule<Completion> { | ||
@Override | ||
protected LogicalPlan rule(Completion p) { | ||
return PushDownUtils.pushGeneratingPlanPastProjectAndOrderBy(p); | ||
} | ||
} |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
159 changes: 159 additions & 0 deletions
159
...a/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineLimitsTests.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,159 @@ | ||
/* | ||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one | ||
* or more contributor license agreements. Licensed under the Elastic License | ||
* 2.0; you may not use this file except in compliance with the Elastic License | ||
* 2.0. | ||
*/ | ||
|
||
package org.elasticsearch.xpack.esql.optimizer.rules.logical; | ||
|
||
import org.elasticsearch.test.ESTestCase; | ||
import org.elasticsearch.xpack.esql.core.expression.Alias; | ||
import org.elasticsearch.xpack.esql.core.expression.Attribute; | ||
import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; | ||
import org.elasticsearch.xpack.esql.core.expression.Literal; | ||
import org.elasticsearch.xpack.esql.expression.Order; | ||
import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToInteger; | ||
import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; | ||
import org.elasticsearch.xpack.esql.plan.logical.EsRelation; | ||
import org.elasticsearch.xpack.esql.plan.logical.Eval; | ||
import org.elasticsearch.xpack.esql.plan.logical.Filter; | ||
import org.elasticsearch.xpack.esql.plan.logical.Limit; | ||
import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; | ||
import org.elasticsearch.xpack.esql.plan.logical.OrderBy; | ||
import org.elasticsearch.xpack.esql.plan.logical.UnaryPlan; | ||
import org.elasticsearch.xpack.esql.plan.logical.inference.Completion; | ||
|
||
import java.util.List; | ||
import java.util.function.BiConsumer; | ||
import java.util.function.BiFunction; | ||
|
||
import static org.elasticsearch.xpack.esql.EsqlTestUtils.as; | ||
import static org.elasticsearch.xpack.esql.EsqlTestUtils.getFieldAttribute; | ||
import static org.elasticsearch.xpack.esql.EsqlTestUtils.randomLiteral; | ||
import static org.elasticsearch.xpack.esql.EsqlTestUtils.unboundLogicalOptimizerContext; | ||
import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY; | ||
import static org.elasticsearch.xpack.esql.core.type.DataType.INTEGER; | ||
import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT; | ||
import static org.elasticsearch.xpack.esql.optimizer.LocalLogicalPlanOptimizerTests.relation; | ||
|
||
public class PushDownAndCombineLimitsTests extends ESTestCase { | ||
|
||
private static class PushDownLimitTestCase<PlanType extends UnaryPlan> { | ||
private final Class<PlanType> clazz; | ||
private final BiFunction<LogicalPlan, Attribute, PlanType> planBuilder; | ||
private final BiConsumer<PlanType, PlanType> planChecker; | ||
|
||
PushDownLimitTestCase( | ||
Class<PlanType> clazz, | ||
BiFunction<LogicalPlan, Attribute, PlanType> planBuilder, | ||
BiConsumer<PlanType, PlanType> planChecker | ||
) { | ||
this.clazz = clazz; | ||
this.planBuilder = planBuilder; | ||
this.planChecker = planChecker; | ||
} | ||
|
||
public PlanType buildPlan(LogicalPlan child, Attribute attr) { | ||
return planBuilder.apply(child, attr); | ||
} | ||
|
||
public void checkOptimizedPlan(LogicalPlan basePlan, LogicalPlan optimizedPlan) { | ||
planChecker.accept(as(basePlan, clazz), as(optimizedPlan, clazz)); | ||
} | ||
} | ||
|
||
private static final List<PushDownLimitTestCase<? extends UnaryPlan>> PUSHABLE_LIMIT_TEST_CASES = List.of( | ||
new PushDownLimitTestCase<>( | ||
Eval.class, | ||
(plan, attr) -> new Eval(EMPTY, plan, List.of(new Alias(EMPTY, "y", new ToInteger(EMPTY, attr)))), | ||
(basePlan, optimizedPlan) -> { | ||
assertEquals(basePlan.source(), optimizedPlan.source()); | ||
assertEquals(basePlan.fields(), optimizedPlan.fields()); | ||
} | ||
), | ||
new PushDownLimitTestCase<>( | ||
Completion.class, | ||
(plan, attr) -> new Completion(EMPTY, plan, randomLiteral(TEXT), randomLiteral(TEXT), attr), | ||
(basePlan, optimizedPlan) -> { | ||
assertEquals(basePlan.source(), optimizedPlan.source()); | ||
assertEquals(basePlan.inferenceId(), optimizedPlan.inferenceId()); | ||
assertEquals(basePlan.prompt(), optimizedPlan.prompt()); | ||
assertEquals(basePlan.targetField(), optimizedPlan.targetField()); | ||
} | ||
) | ||
); | ||
|
||
private static final List<PushDownLimitTestCase<? extends UnaryPlan>> NON_PUSHABLE_LIMIT_TEST_CASES = List.of( | ||
new PushDownLimitTestCase<>( | ||
Filter.class, | ||
(plan, attr) -> new Filter(EMPTY, plan, new Equals(EMPTY, attr, new Literal(EMPTY, "right", TEXT))), | ||
(basePlan, optimizedPlan) -> { | ||
assertEquals(basePlan.source(), optimizedPlan.source()); | ||
assertEquals(basePlan.condition(), optimizedPlan.condition()); | ||
} | ||
), | ||
new PushDownLimitTestCase<>( | ||
OrderBy.class, | ||
(plan, attr) -> new OrderBy(EMPTY, plan, List.of(new Order(EMPTY, attr, Order.OrderDirection.DESC, null))), | ||
(basePlan, optimizedPlan) -> { | ||
assertEquals(basePlan.source(), optimizedPlan.source()); | ||
assertEquals(basePlan.order(), optimizedPlan.order()); | ||
} | ||
) | ||
); | ||
|
||
public void testPushableLimit() { | ||
FieldAttribute a = getFieldAttribute("a"); | ||
FieldAttribute b = getFieldAttribute("b"); | ||
EsRelation relation = relation().withAttributes(List.of(a, b)); | ||
|
||
for (PushDownLimitTestCase<? extends UnaryPlan> pushableLimitTestCase : PUSHABLE_LIMIT_TEST_CASES) { | ||
int precedingLimitValue = randomIntBetween(1, 10_000); | ||
Limit precedingLimit = new Limit(EMPTY, new Literal(EMPTY, precedingLimitValue, INTEGER), relation); | ||
|
||
LogicalPlan pushableLimitTestPlan = pushableLimitTestCase.buildPlan(precedingLimit, a); | ||
|
||
int pushableLimitValue = randomIntBetween(1, 10_000); | ||
Limit pushableLimit = new Limit(EMPTY, new Literal(EMPTY, pushableLimitValue, INTEGER), pushableLimitTestPlan); | ||
|
||
LogicalPlan optimizedPlan = optimizePlan(pushableLimit); | ||
|
||
pushableLimitTestCase.checkOptimizedPlan(pushableLimitTestPlan, optimizedPlan); | ||
|
||
assertEquals( | ||
as(optimizedPlan, UnaryPlan.class).child(), | ||
new Limit(EMPTY, new Literal(EMPTY, Math.min(pushableLimitValue, precedingLimitValue), INTEGER), relation) | ||
); | ||
} | ||
} | ||
|
||
public void testNonPushableLimit() { | ||
FieldAttribute a = getFieldAttribute("a"); | ||
FieldAttribute b = getFieldAttribute("b"); | ||
EsRelation relation = relation().withAttributes(List.of(a, b)); | ||
|
||
for (PushDownLimitTestCase<? extends UnaryPlan> nonPushableLimitTestCase : NON_PUSHABLE_LIMIT_TEST_CASES) { | ||
int precedingLimitValue = randomIntBetween(1, 10_000); | ||
Limit precedingLimit = new Limit(EMPTY, new Literal(EMPTY, precedingLimitValue, INTEGER), relation); | ||
UnaryPlan nonPushableLimitTestPlan = nonPushableLimitTestCase.buildPlan(precedingLimit, a); | ||
int nonPushableLimitValue = randomIntBetween(1, 10_000); | ||
Limit nonPushableLimit = new Limit(EMPTY, new Literal(EMPTY, nonPushableLimitValue, INTEGER), nonPushableLimitTestPlan); | ||
Limit optimizedPlan = as(optimizePlan(nonPushableLimit), Limit.class); | ||
nonPushableLimitTestCase.checkOptimizedPlan(nonPushableLimitTestPlan, optimizedPlan.child()); | ||
assertEquals( | ||
optimizedPlan, | ||
new Limit( | ||
EMPTY, | ||
new Literal(EMPTY, Math.min(nonPushableLimitValue, precedingLimitValue), INTEGER), | ||
nonPushableLimitTestPlan | ||
) | ||
); | ||
assertEquals(as(optimizedPlan.child(), UnaryPlan.class).child(), nonPushableLimitTestPlan.child()); | ||
} | ||
} | ||
|
||
private LogicalPlan optimizePlan(LogicalPlan plan) { | ||
return new PushDownAndCombineLimits().apply(plan, unboundLogicalOptimizerContext()); | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I might have missed this - is the
PushDownAndCombineLimits
change being tested?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added some test for
PushDownAndCombineLimits
(it is wild but none were created for other commands)There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The
LogicalPlanOptimizerTests
havetestCombineLimits
andtestMultipleCombineLimits
, but indeed, I don't immediately see tests for the case where there are compatible plans between the two limits. Thanks a lot for the added tests.