diff --git a/.idea/runConfigurations/Debug_Elasticsearch__node_2_.xml b/.idea/runConfigurations/Debug_Elasticsearch__node_2_.xml
index 94bb079398ffd..76bb8f72eafc9 100644
--- a/.idea/runConfigurations/Debug_Elasticsearch__node_2_.xml
+++ b/.idea/runConfigurations/Debug_Elasticsearch__node_2_.xml
@@ -6,6 +6,10 @@
+
+
+
+
\ No newline at end of file
diff --git a/.idea/runConfigurations/Debug_Elasticsearch__node_3_.xml b/.idea/runConfigurations/Debug_Elasticsearch__node_3_.xml
index aaef20fec729b..a321a38f14e57 100644
--- a/.idea/runConfigurations/Debug_Elasticsearch__node_3_.xml
+++ b/.idea/runConfigurations/Debug_Elasticsearch__node_3_.xml
@@ -6,6 +6,10 @@
+
+
+
+
\ No newline at end of file
diff --git a/docs/changelog/133166.yaml b/docs/changelog/133166.yaml
new file mode 100644
index 0000000000000..eadd1b2bad8be
--- /dev/null
+++ b/docs/changelog/133166.yaml
@@ -0,0 +1,6 @@
+pr: 133166
+summary: Improve Expanding Lookup Join performance by pushing a filter to the right
+ side of the lookup join
+area: ES|QL
+type: enhancement
+issues: []
diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java
index da61d44264571..8ce4bbe95c893 100644
--- a/server/src/main/java/org/elasticsearch/TransportVersions.java
+++ b/server/src/main/java/org/elasticsearch/TransportVersions.java
@@ -357,6 +357,8 @@ static TransportVersion def(int id) {
public static final TransportVersion ESQL_SAMPLE_OPERATOR_STATUS = def(9_127_0_00);
public static final TransportVersion ALLOCATION_DECISION_NOT_PREFERRED = def(9_145_0_00);
public static final TransportVersion ESQL_QUALIFIERS_IN_ATTRIBUTES = def(9_146_0_00);
+ public static final TransportVersion ESQL_LOOKUP_JOIN_PRE_JOIN_FILTER = def(9_147_0_00);
+ public static final TransportVersion ESQL_LOOKUP_JOIN_ON_EXPRESSION = def(9_148_0_00);
/*
* STOP! READ THIS FIRST! No, really,
diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java
index 24be16d3ee6df..77a13865f8d7c 100644
--- a/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java
+++ b/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java
@@ -387,7 +387,7 @@ public Query regexpQuery(
}
/**
- * Returns a Lucine pushable Query for the current field
+ * Returns a Lucene pushable Query for the current field
* For now can only be AutomatonQuery or MatchAllDocsQuery() or MatchNoDocsQuery()
*/
public Query automatonQuery(
diff --git a/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java b/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java
index b463202221dc1..2f7b21fa3564b 100644
--- a/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java
+++ b/test/external-modules/esql-heap-attack/src/javaRestTest/java/org/elasticsearch/xpack/esql/heap_attack/HeapAttackIT.java
@@ -698,7 +698,7 @@ private Map fetchMvLongs() throws IOException {
public void testLookupExplosion() throws IOException {
int sensorDataCount = 400;
int lookupEntries = 10000;
- Map, ?> map = lookupExplosion(sensorDataCount, lookupEntries, 1);
+ Map, ?> map = lookupExplosion(sensorDataCount, lookupEntries, 1, lookupEntries);
assertMap(map, matchesMap().extraOk().entry("values", List.of(List.of(sensorDataCount * lookupEntries))));
}
@@ -706,18 +706,34 @@ public void testLookupExplosionManyFields() throws IOException {
int sensorDataCount = 400;
int lookupEntries = 1000;
int joinFieldsCount = 990;
- Map, ?> map = lookupExplosion(sensorDataCount, lookupEntries, joinFieldsCount);
+ Map, ?> map = lookupExplosion(sensorDataCount, lookupEntries, joinFieldsCount, lookupEntries);
assertMap(map, matchesMap().extraOk().entry("values", List.of(List.of(sensorDataCount * lookupEntries))));
}
public void testLookupExplosionManyMatchesManyFields() throws IOException {
// 1500, 10000 is enough locally, but some CI machines need more.
- assertCircuitBreaks(attempt -> lookupExplosion(attempt * 1500, 10000, 30));
+ int lookupEntries = 10000;
+ assertCircuitBreaks(attempt -> lookupExplosion(attempt * 1500, lookupEntries, 30, lookupEntries));
}
public void testLookupExplosionManyMatches() throws IOException {
// 1500, 10000 is enough locally, but some CI machines need more.
- assertCircuitBreaks(attempt -> lookupExplosion(attempt * 1500, 10000, 1));
+ int lookupEntries = 10000;
+ assertCircuitBreaks(attempt -> lookupExplosion(attempt * 1500, lookupEntries, 1, lookupEntries));
+ }
+
+ public void testLookupExplosionManyMatchesFiltered() throws IOException {
+ // This test will only work with the expanding join optimization
+ // that pushes the filter to the right side of the lookup.
+ // Without the optimization, it will fail with circuit_breaking_exception
+ int sensorDataCount = 10000;
+ int lookupEntries = 10000;
+ int reductionFactor = 1000; // reduce the number of matches by this factor
+ // lookupEntries % reductionFactor must be 0 to ensure the number of rows returned matches the expected value
+ assertTrue(0 == lookupEntries % reductionFactor);
+ Map, ?> map = lookupExplosion(sensorDataCount, lookupEntries, 1, lookupEntries / reductionFactor);
+ assertMap(map, matchesMap().extraOk().entry("values", List.of(List.of(sensorDataCount * lookupEntries / reductionFactor))));
+
}
public void testLookupExplosionNoFetch() throws IOException {
@@ -744,7 +760,8 @@ public void testLookupExplosionBigStringManyMatches() throws IOException {
assertCircuitBreaks(attempt -> lookupExplosionBigString(attempt * 500, 1));
}
- private Map lookupExplosion(int sensorDataCount, int lookupEntries, int joinFieldsCount) throws IOException {
+ private Map lookupExplosion(int sensorDataCount, int lookupEntries, int joinFieldsCount, int lookupEntriesToKeep)
+ throws IOException {
try {
lookupExplosionData(sensorDataCount, lookupEntries, joinFieldsCount);
StringBuilder query = startQuery();
@@ -755,7 +772,14 @@ private Map lookupExplosion(int sensorDataCount, int lookupEntri
}
query.append("id").append(i);
}
- query.append(" | STATS COUNT(location)\"}");
+ if (lookupEntries != lookupEntriesToKeep) {
+ // add a filter to reduce the number of matches
+ // we add both a Lucene pushable filter and a non-pushable filter
+ // this is to make sure that even if there are non-pushable filters the pushable filters is still applied
+ query.append(" | WHERE ABS(filter_key) > -1 AND filter_key < ").append(lookupEntriesToKeep);
+
+ }
+ query.append(" | STATS COUNT(location) | LIMIT 100\"}");
return responseAsMap(query(query.toString(), null));
} finally {
deleteIndex("sensor_data");
@@ -1038,7 +1062,8 @@ private void initSensorLookup(int lookupEntries, int sensorCount, IntFunction queryLists;
-
- public ExpressionQueryList(List queryLists) {
- if (queryLists.size() < 2) {
- throw new IllegalArgumentException("ExpressionQueryList must have at least two QueryLists");
- }
- this.queryLists = queryLists;
- }
-
- @Override
- public Query getQuery(int position) {
- BooleanQuery.Builder builder = new BooleanQuery.Builder();
- for (QueryList queryList : queryLists) {
- Query q = queryList.getQuery(position);
- if (q == null) {
- // if any of the matchFields are null, it means there is no match for this position
- // A AND NULL is always NULL, so we can skip this position
- return null;
- }
- builder.add(q, BooleanClause.Occur.FILTER);
- }
- return builder.build();
- }
-
- @Override
- public int getPositionCount() {
- int positionCount = queryLists.get(0).getPositionCount();
- for (QueryList queryList : queryLists) {
- if (queryList.getPositionCount() != positionCount) {
- throw new IllegalStateException(
- "All QueryLists must have the same position count, expected: "
- + positionCount
- + ", but got: "
- + queryList.getPositionCount()
- );
- }
- }
- return positionCount;
- }
-}
diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/QueryList.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/QueryList.java
index 4d93ab63aaa5d..93acfe8ba37cd 100644
--- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/QueryList.java
+++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/QueryList.java
@@ -125,7 +125,7 @@ public final Query getQuery(int position) {
* Returns the query at the given position.
*/
@Nullable
- abstract Query doGetQuery(int position, int firstValueIndex, int valueCount);
+ public abstract Query doGetQuery(int position, int firstValueIndex, int valueCount);
private Query wrapSingleValueQuery(Query query) {
assert onlySingleValueParams != null : "Requested to wrap single value query without single value params";
@@ -159,13 +159,8 @@ private Query wrapSingleValueQuery(Query query) {
* using only the {@link ElementType} of the {@link Block} to determine the
* query.
*/
- public static QueryList rawTermQueryList(
- MappedFieldType field,
- SearchExecutionContext searchExecutionContext,
- AliasFilter aliasFilter,
- Block block
- ) {
- IntFunction blockToJavaObject = switch (block.elementType()) {
+ public static IntFunction createBlockValueReader(Block block) {
+ return switch (block.elementType()) {
case BOOLEAN -> {
BooleanBlock booleanBlock = (BooleanBlock) block;
yield booleanBlock::getBoolean;
@@ -196,7 +191,20 @@ public static QueryList rawTermQueryList(
case AGGREGATE_METRIC_DOUBLE -> throw new IllegalArgumentException("can't read values from [aggregate metric double] block");
case UNKNOWN -> throw new IllegalArgumentException("can't read values from [" + block + "]");
};
- return new TermQueryList(field, searchExecutionContext, aliasFilter, block, null, blockToJavaObject);
+ }
+
+ /**
+ * Returns a list of term queries for the given field and the input block
+ * using only the {@link ElementType} of the {@link Block} to determine the
+ * query.
+ */
+ public static QueryList rawTermQueryList(
+ MappedFieldType field,
+ SearchExecutionContext searchExecutionContext,
+ AliasFilter aliasFilter,
+ Block block
+ ) {
+ return new TermQueryList(field, searchExecutionContext, aliasFilter, block, null, createBlockValueReader(block));
}
/**
@@ -297,7 +305,7 @@ public TermQueryList onlySingleValues(Warnings warnings, String multiValueWarnin
}
@Override
- Query doGetQuery(int position, int firstValueIndex, int valueCount) {
+ public Query doGetQuery(int position, int firstValueIndex, int valueCount) {
return switch (valueCount) {
case 0 -> null;
case 1 -> field.termQuery(blockValueReader.apply(firstValueIndex), searchExecutionContext);
@@ -360,7 +368,7 @@ public DateNanosQueryList onlySingleValues(Warnings warnings, String multiValueW
}
@Override
- Query doGetQuery(int position, int firstValueIndex, int valueCount) {
+ public Query doGetQuery(int position, int firstValueIndex, int valueCount) {
return switch (valueCount) {
case 0 -> null;
case 1 -> dateFieldType.equalityQuery(blockValueReader.apply(firstValueIndex), searchExecutionContext);
@@ -412,7 +420,7 @@ public GeoShapeQueryList onlySingleValues(Warnings warnings, String multiValueWa
}
@Override
- Query doGetQuery(int position, int firstValueIndex, int valueCount) {
+ public Query doGetQuery(int position, int firstValueIndex, int valueCount) {
return switch (valueCount) {
case 0 -> null;
case 1 -> shapeQuery.apply(firstValueIndex);
@@ -453,5 +461,5 @@ private IntFunction shapeQuery() {
}
}
- protected record OnlySingleValueParams(Warnings warnings, String multiValueWarningMessage) {}
+ public record OnlySingleValueParams(Warnings warnings, String multiValueWarningMessage) {}
}
diff --git a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/AnyOperatorTestCase.java b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/AnyOperatorTestCase.java
index e1a2110111d1e..5bffb1c61f30e 100644
--- a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/AnyOperatorTestCase.java
+++ b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/AnyOperatorTestCase.java
@@ -87,7 +87,7 @@ protected final Operator.OperatorFactory simple() {
/**
* Makes sure the description of {@link #simple} matches the {@link #expectedDescriptionOfSimple}.
*/
- public final void testSimpleDescription() {
+ public void testSimpleDescription() {
Operator.OperatorFactory factory = simple();
String description = factory.describe();
assertThat(description, expectedDescriptionOfSimple());
diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join-expression.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join-expression.csv-spec
new file mode 100644
index 0000000000000..6b29a6db34b5f
--- /dev/null
+++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join-expression.csv-spec
@@ -0,0 +1,515 @@
+//
+// CSV spec for LOOKUP JOIN command with expression join
+//
+
+lookupWithExpressionEquals
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM employees
+| WHERE emp_no == 10001
+| LOOKUP JOIN languages_lookup ON languages == language_code
+| KEEP emp_no, languages, language_code, language_name
+| SORT emp_no, languages, language_code, language_name
+;
+
+emp_no:integer | languages:integer | language_code:integer | language_name:keyword
+10001 | 2 | 2 | French
+;
+
+lookupWithExpressionNotEquals
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM employees
+| WHERE emp_no == 10001
+| LOOKUP JOIN languages_lookup ON languages != language_code
+| KEEP emp_no, languages, language_code, language_name
+| SORT emp_no, languages, language_code, language_name
+;
+
+emp_no:integer | languages:integer | language_code:integer | language_name:keyword
+10001 | 2 | 1 | English
+10001 | 2 | 3 | Spanish
+10001 | 2 | 4 | German
+;
+
+lookupWithExpressionGreater
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM employees
+| WHERE emp_no == 10001
+| LOOKUP JOIN languages_lookup ON languages > language_code
+| KEEP emp_no, languages, language_code, language_name
+| SORT emp_no, languages, language_code, language_name
+;
+
+emp_no:integer | languages:integer | language_code:integer | language_name:keyword
+10001 | 2 | 1 | English
+;
+
+lookupWithExpressionGreaterOrEquals
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM employees
+| WHERE emp_no == 10001
+| LOOKUP JOIN languages_lookup ON languages >= language_code
+| KEEP emp_no, languages, language_code, language_name
+| SORT emp_no, languages, language_code, language_name
+;
+
+emp_no:integer | languages:integer | language_code:integer | language_name:keyword
+10001 | 2 | 1 | English
+10001 | 2 | 2 | French
+;
+
+lookupWithExpressionLess
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM employees
+| WHERE emp_no == 10001
+| LOOKUP JOIN languages_lookup ON languages < language_code
+| KEEP emp_no, languages, language_code, language_name
+| SORT emp_no, languages, language_code, language_name
+;
+
+emp_no:integer | languages:integer | language_code:integer | language_name:keyword
+10001 | 2 | 3 | Spanish
+10001 | 2 | 4 | German
+;
+
+lookupWithExpressionLessOrEquals
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM employees
+| WHERE emp_no == 10001
+| LOOKUP JOIN languages_lookup ON languages <= language_code
+| KEEP emp_no, languages, language_code, language_name
+| SORT emp_no, languages, language_code, language_name
+;
+
+emp_no:integer | languages:integer | language_code:integer | language_name:keyword
+10001 | 2 | 2 | French
+10001 | 2 | 3 | Spanish
+10001 | 2 | 4 | German
+;
+
+lookupJoinOnTwoFieldsWithEval
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| eval id_int = id_int + 5
+| RENAME id_int AS id_left, is_active_bool AS is_active_left
+| LOOKUP JOIN multi_column_joinable_lookup ON id_left == id_int AND is_active_left == is_active_bool
+| KEEP id_left, name_str, extra1, other1, other2
+| SORT id_left, name_str, extra1, other1, other2
+;
+
+warning:Line 2:17: evaluation of [id_int + 5] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:17: java.lang.IllegalArgumentException: single-value function encountered multi-value
+warning:Line 4:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_left == id_int AND is_active_left == is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 4:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_left:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+6 | null | foo | iota | 9000
+7 | Grace | bar | kappa | 10000
+8 | Hank | baz | lambda | 11000
+9 | null | qux | null | null
+10 | null | quux | null | null
+11 | null | corge | null | null
+12 | null | grault | null | null
+13 | null | garply | null | null
+14 | null | waldo | null | null
+15 | null | fred | null | null
+17 | null | xyzzy | null | null
+18 | null | thud | null | null
+19 | null | foo2 | null | null
+20 | null | bar2 | null | null
+null | null | plugh | null | null
+null | null | xyz | null | null
+null | null | zyx | null | null
+;
+
+lookupWithTwoExpressionsAndNoMatch
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM employees
+| WHERE emp_no == 10001
+| EVAL expected_lang = "French"
+| LOOKUP JOIN languages_lookup ON languages > language_code AND expected_lang == language_name
+| KEEP emp_no, languages, language_code, language_name
+| SORT emp_no, languages, language_code, language_name
+;
+
+emp_no:integer | languages:integer | language_code:integer | language_name:keyword
+10001 | 2 | null | null
+;
+
+lookupWithTwoExpressionsSelfJoin
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM languages_lookup
+| RENAME language_code AS left_code, language_name AS left_name
+| LOOKUP JOIN languages_lookup ON left_code > language_code AND left_name != language_name
+| WHERE left_code == 2
+| KEEP left_code, left_name, language_code, language_name
+| SORT left_code, left_name, language_code, language_name
+;
+
+left_code:integer | left_name:keyword | language_code:integer | language_name:keyword
+2 | French | 1 | English
+;
+
+lookupMultiSelfJoinColTwoExprAnd
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable_lookup
+| RENAME id_int AS left_id, name_str AS left_name
+| LOOKUP JOIN multi_column_joinable_lookup ON left_id == id_int AND left_name == name_str
+| KEEP left_id, left_name, id_int, name_str
+| SORT left_id, left_name, id_int, name_str
+;
+
+warning:Line 3:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON left_id == id_int AND left_name == name_str] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+left_id:integer | left_name:keyword | id_int:integer | name_str:keyword
+1 | Alice | 1 | Alice
+1 | Alice | 1 | Alice
+1 | Alice | 1 | Alice
+1 | Alice | 1 | Alice
+[1, 19, 20] | Sophia | null | null
+2 | Bob | 2 | Bob
+3 | Charlie | 3 | Charlie
+3 | Charlie | 3 | Charlie
+3 | Charlie | 3 | Charlie
+3 | Charlie | 3 | Charlie
+4 | David | 4 | David
+5 | Eve | 5 | Eve
+5 | Eve | 5 | Eve
+5 | Eve | 5 | Eve
+5 | Eve | 5 | Eve
+6 | null | null | null
+7 | Grace | 7 | Grace
+8 | Hank | 8 | Hank
+12 | Liam | 12 | Liam
+13 | Mia | 13 | Mia
+14 | Nina | 14 | Nina
+16 | Paul | 16 | Paul
+[17, 18] | Olivia | null | null
+null | Kate | null | null
+;
+
+lookupJoinOnThreeFields
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| RENAME id_int as id_left, name_str as name_left, is_active_bool as is_active_left
+| LOOKUP JOIN multi_column_joinable_lookup ON id_left == id_int AND name_left == name_str AND is_active_left == is_active_bool
+| KEEP id_left, name_left, extra1, other1, other2
+| SORT id_left, name_left, extra1, other1, other2
+;
+
+warning:Line 3:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_left == id_int AND name_left == name_str AND is_active_left == is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_left:integer| name_left:keyword| extra1:keyword | other1:keyword | other2:integer
+1 | Alice | foo | alpha | 1000
+1 | Alice | foo | beta | 2000
+[1, 19, 21] | Sophia | zyx | null | null
+2 | Bob | bar | gamma | 3000
+3 | Charlie | baz | delta | 4000
+4 | David | qux | zeta | 6000
+5 | Eve | quux | eta | 7000
+5 | Eve | quux | theta | 8000
+6 | null | corge | null | null
+7 | Grace | grault | kappa | 10000
+8 | Hank | garply | lambda | 11000
+9 | Ivy | waldo | null | null
+10 | John | fred | null | null
+12 | Liam | xyzzy | nu | 13000
+13 | Mia | thud | xi | 14000
+14 | Nina | foo2 | omicron | 15000
+15 | Oscar | bar2 | null | null
+[17, 18] | Olivia | xyz | null | null
+null | Kate | plugh | null | null
+;
+
+lookupJoinOnFourFields
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| RENAME id_int AS id_left, name_str AS name_left, is_active_bool AS is_active_left, ip_addr AS ip_addr_left
+| LOOKUP JOIN multi_column_joinable_lookup ON id_left == id_int AND name_left == name_str AND is_active_left == is_active_bool AND ip_addr_left == ip_addr
+| KEEP id_left, name_left, extra1, other1, other2
+| SORT id_left, name_left, extra1, other1, other2
+;
+
+warning:Line 3:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_left == id_int AND name_left == name_str AND is_active_left == is_active_bool AND ip_addr_left == ip_addr] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_left:integer| name_left:keyword | extra1:keyword | other1:keyword | other2:integer
+1 | Alice | foo | alpha | 1000
+[1, 19, 21] | Sophia | zyx | null | null
+2 | Bob | bar | null | null
+3 | Charlie | baz | delta | 4000
+4 | David | qux | zeta | 6000
+5 | Eve | quux | eta | 7000
+5 | Eve | quux | theta | 8000
+6 | null | corge | null | null
+7 | Grace | grault | null | null
+8 | Hank | garply | lambda | 11000
+9 | Ivy | waldo | null | null
+10 | John | fred | null | null
+12 | Liam | xyzzy | nu | 13000
+13 | Mia | thud | xi | 14000
+14 | Nina | foo2 | omicron | 15000
+15 | Oscar | bar2 | null | null
+[17, 18] | Olivia | xyz | null | null
+null | Kate | plugh | null | null
+;
+
+
+lookupMultiColTwoExprAndNoMatch
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable_lookup
+| RENAME id_int AS left_id, name_str AS left_name
+| EVAL left_name = "nomatch"
+| LOOKUP JOIN multi_column_joinable_lookup ON left_id == id_int AND left_name == name_str
+| KEEP left_id, left_name, id_int, name_str
+| SORT left_id, left_name, id_int, name_str
+;
+
+warning:Line 4:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON left_id == id_int AND left_name == name_str] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 4:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+left_id:integer | left_name:keyword | id_int:integer | name_str:keyword
+[1, 19, 20] | nomatch | null | null
+1 | nomatch | null | null
+1 | nomatch | null | null
+2 | nomatch | null | null
+3 | nomatch | null | null
+3 | nomatch | null | null
+4 | nomatch | null | null
+5 | nomatch | null | null
+5 | nomatch | null | null
+6 | nomatch | null | null
+7 | nomatch | null | null
+8 | nomatch | null | null
+12 | nomatch | null | null
+13 | nomatch | null | null
+14 | nomatch | null | null
+16 | nomatch | null | null
+[17, 18] | nomatch | null | null
+null | nomatch | null | null
+;
+
+lookupMultiColMixedEqNeq
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable
+| RENAME name_str AS left_name, is_active_bool AS is_active_left, id_int AS left_id
+| LOOKUP JOIN multi_column_joinable_lookup ON is_active_left == is_active_bool AND left_id != id_int
+| KEEP left_id, left_name, id_int, name_str
+| SORT left_id, left_name, id_int, name_str
+| LIMIT 20
+;
+
+warning:Line 3:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON is_active_left == is_active_bool AND left_id != id_int] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+left_id:integer | left_name:keyword | id_int:integer | name_str:keyword
+1 | Alice | 3 | Charlie
+1 | Alice | 5 | Eve
+1 | Alice | 5 | Eve
+1 | Alice | 6 | null
+1 | Alice | 8 | Hank
+1 | Alice | 12 | Liam
+1 | Alice | 14 | Nina
+1 | Alice | 16 | Paul
+[1, 19, 21] | Sophia | null | null
+2 | Bob | 3 | Charlie
+2 | Bob | 4 | David
+2 | Bob | 7 | Grace
+2 | Bob | 13 | Mia
+3 | Charlie | 1 | Alice
+3 | Charlie | 1 | Alice
+3 | Charlie | 5 | Eve
+3 | Charlie | 5 | Eve
+3 | Charlie | 6 | null
+3 | Charlie | 8 | Hank
+3 | Charlie | 12 | Liam
+;
+
+lookupMultiColMixedGtEq
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable
+| RENAME id_int AS left_id, name_str AS left_name, is_active_bool AS is_active_left
+| LOOKUP JOIN multi_column_joinable_lookup ON left_id > id_int AND is_active_left == is_active_bool
+| KEEP left_id, left_name, id_int, name_str, is_active_left, is_active_bool
+| SORT left_id, left_name, id_int, name_str, is_active_left, is_active_left
+| LIMIT 20
+;
+
+warning:Line 3:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON left_id > id_int AND is_active_left == is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+
+left_id:integer | left_name:keyword | id_int:integer | name_str:keyword | is_active_left:boolean | is_active_bool:boolean
+1 | Alice | null | null | true | null
+[1, 19, 21] | Sophia | null | null | true | null
+2 | Bob | null | null | false | null
+3 | Charlie | 1 | Alice | true | true
+3 | Charlie | 1 | Alice | true | true
+4 | David | 2 | Bob | false | false
+4 | David | 3 | Charlie | false | false
+5 | Eve | 1 | Alice | true | true
+5 | Eve | 1 | Alice | true | true
+5 | Eve | 3 | Charlie | true | true
+6 | null | 1 | Alice | true | true
+6 | null | 1 | Alice | true | true
+6 | null | 3 | Charlie | true | true
+6 | null | 5 | Eve | true | true
+6 | null | 5 | Eve | true | true
+7 | Grace | 2 | Bob | false | false
+7 | Grace | 3 | Charlie | false | false
+7 | Grace | 4 | David | false | false
+8 | Hank | 1 | Alice | true | true
+8 | Hank | 1 | Alice | true | true
+;
+
+lookupMultiColMixedLtNeqEq
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable
+| RENAME id_int AS left_id, name_str AS left_name, is_active_bool AS left_is_active
+| LOOKUP JOIN multi_column_joinable_lookup ON left_id < id_int AND left_name != name_str AND left_is_active == is_active_bool
+| KEEP left_id, left_name, left_is_active, id_int, name_str, is_active_bool
+| SORT left_id, left_name, left_is_active, id_int, name_str, is_active_bool
+| LIMIT 20
+;
+
+warning:Line 3:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON left_id < id_int AND left_name != name_str AND left_is_active == is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+left_id:integer | left_name:keyword | left_is_active:boolean | id_int:integer | name_str:keyword | is_active_bool:boolean
+1 | Alice | true | 3 | Charlie | true
+1 | Alice | true | 5 | Eve | true
+1 | Alice | true | 5 | Eve | true
+1 | Alice | true | 8 | Hank | true
+1 | Alice | true | 12 | Liam | true
+1 | Alice | true | 14 | Nina | true
+1 | Alice | true | 16 | Paul | true
+[1, 19, 21] | Sophia | true | null | null | null
+2 | Bob | false | 3 | Charlie | false
+2 | Bob | false | 4 | David | false
+2 | Bob | false | 7 | Grace | false
+2 | Bob | false | 13 | Mia | false
+3 | Charlie | true | 5 | Eve | true
+3 | Charlie | true | 5 | Eve | true
+3 | Charlie | true | 8 | Hank | true
+3 | Charlie | true | 12 | Liam | true
+3 | Charlie | true | 14 | Nina | true
+3 | Charlie | true | 16 | Paul | true
+4 | David | false | 7 | Grace | false
+4 | David | false | 13 | Mia | false
+;
+
+lookupMultiColMixedGteNeq
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable
+| RENAME id_int AS left_id, name_str AS left_name
+| LOOKUP JOIN multi_column_joinable_lookup ON left_id >= id_int AND left_name != name_str
+| KEEP left_id, left_name, id_int, name_str
+| SORT left_id, left_name, id_int, name_str
+| LIMIT 20
+;
+
+warning:Line 3:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON left_id >= id_int AND left_name != name_str] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+left_id:integer | left_name:keyword | id_int:integer | name_str:keyword
+1 | Alice | null | null
+[1, 19, 21] | Sophia | null | null
+2 | Bob | 1 | Alice
+2 | Bob | 1 | Alice
+3 | Charlie | 1 | Alice
+3 | Charlie | 1 | Alice
+3 | Charlie | 2 | Bob
+4 | David | 1 | Alice
+4 | David | 1 | Alice
+4 | David | 2 | Bob
+4 | David | 3 | Charlie
+4 | David | 3 | Charlie
+5 | Eve | 1 | Alice
+5 | Eve | 1 | Alice
+5 | Eve | 2 | Bob
+5 | Eve | 3 | Charlie
+5 | Eve | 3 | Charlie
+5 | Eve | 4 | David
+6 | null | null | null
+7 | Grace | 1 | Alice
+;
+
+lookupMultiColMixedLteEq
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable_lookup
+| WHERE id_int == 3
+| RENAME id_int AS left_id, name_str AS left_name
+| LOOKUP JOIN multi_column_joinable_lookup ON left_id <= id_int AND left_name == name_str
+| KEEP left_id, left_name, id_int, name_str
+| SORT left_id, left_name, id_int, name_str
+;
+
+
+left_id:integer | left_name:keyword | id_int:integer | name_str:keyword
+3 | Charlie | 3 | Charlie
+3 | Charlie | 3 | Charlie
+3 | Charlie | 3 | Charlie
+3 | Charlie | 3 | Charlie
+;
+
+lookupMultiColFourMixed
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_boolean_expression
+
+FROM multi_column_joinable_lookup
+| WHERE id_int == 2
+| RENAME id_int AS left_id, name_str AS left_name, is_active_bool AS left_is_active, ip_addr AS left_ip
+| LOOKUP JOIN multi_column_joinable_lookup ON left_id < id_int AND left_name != name_str AND left_is_active != is_active_bool AND left_ip != ip_addr
+| KEEP left_id, left_name, left_is_active, left_ip, id_int, name_str, is_active_bool, ip_addr
+| SORT left_id, left_name, left_is_active, left_ip, id_int, name_str, is_active_bool, ip_addr
+;
+
+warning:Line 4:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON left_id < id_int AND left_name != name_str AND left_is_active != is_active_bool AND left_ip != ip_addr] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 4:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+left_id:integer | left_name:keyword | left_is_active:boolean | left_ip:ip | id_int:integer | name_str:keyword | is_active_bool:boolean | ip_addr:ip
+2 | Bob | false | 192.168.1.3 | 5 | Eve | true | 192.168.1.5
+2 | Bob | false | 192.168.1.3 | 5 | Eve | true | 192.168.1.5
+2 | Bob | false | 192.168.1.3 | 8 | Hank | true | 192.168.1.8
+2 | Bob | false | 192.168.1.3 | 12 | Liam | true | 192.168.1.12
+2 | Bob | false | 192.168.1.3 | 14 | Nina | true | 192.168.1.14
+2 | Bob | false | 192.168.1.3 | 16 | Paul | true | 192.168.1.16
+
+;
diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec
index 8694505411291..45f1cd615e374 100644
--- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec
+++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec
@@ -5149,3 +5149,311 @@ null | null | bar2 | null | null
null | null | corge | null | null
null | null | fred | null | null
;
+
+
+lookupJoinWithPushableFilterOnRight
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+| WHERE other2 > 5000
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int, name_str, extra1, other1, other2
+| LIMIT 20
+;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+4 | David | qux | zeta | 6000
+5 | Eve | quux | eta | 7000
+5 | Eve | quux | theta | 8000
+6 | null | corge | iota | 9000
+7 | Grace | grault | kappa | 10000
+8 | Hank | garply | lambda | 11000
+12 | Liam | xyzzy | nu | 13000
+13 | Mia | thud | xi | 14000
+14 | Nina | foo2 | omicron | 15000
+;
+
+lookupJoinWithPushableFilterOnRightOneField
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int
+| WHERE other2 > 5000
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int, name_str, extra1, other1, other2
+| LIMIT 20
+;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+4 | David | qux | zeta | 6000
+5 | Eve | quux | eta | 7000
+5 | Eve | quux | theta | 8000
+6 | null | corge | iota | 9000
+7 | Grace | grault | kappa | 10000
+8 | Hank | garply | lambda | 11000
+12 | Liam | xyzzy | nu | 13000
+13 | Mia | thud | xi | 14000
+14 | Nina | foo2 | omicron | 15000
+;
+
+lookupJoinWithTwoPushableFiltersOnRight
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+| WHERE other2 > 5000
+| WHERE other1 like "*ta"
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int, name_str, extra1, other1, other2
+| LIMIT 20
+;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+4 | David | qux | zeta | 6000
+5 | Eve | quux | eta | 7000
+5 | Eve | quux | theta | 8000
+6 | null | corge | iota | 9000
+;
+
+lookupJoinWithCoalesceFilterOnRight
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+| WHERE COALESCE(other1, "zeta") == "zeta"
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int, name_str, extra1, other1, other2
+| LIMIT 20
+;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+[1, 19, 21] | null | zyx | null | null
+4 | David | qux | zeta | 6000
+9 | null | waldo | null | null
+10 | null | fred | null | null
+15 | null | bar2 | null | null
+[17, 18] | null | xyz | null | null
+null | null | plugh | null | null
+;
+
+lookupJoinWithIsNullFilterOnRight
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+| WHERE other1 IS NULL
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int
+;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+[1, 19, 21] | null | zyx | null | null
+9 | null | waldo | null | null
+10 | null | fred | null | null
+15 | null | bar2 | null | null
+[17, 18] | null | xyz | null | null
+null | null | plugh | null | null
+;
+
+lookupJoinWithIsNullJoinKey
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON name_str
+| WHERE name_str IS NULL
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int
+;
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+null | null | corge | null | null
+;
+
+lookupJoinWithMixLeftAndRightFilters
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+| WHERE other2 > 5000 AND (extra1 == "qux" OR extra1 == "foo2") AND other1 like ("*ta", "*ron")
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int, name_str, extra1, other1, other2
+| LIMIT 20
+;
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+4 | David | qux | zeta | 6000
+14 | Nina | foo2 | omicron | 15000
+;
+
+lookupJoinWithMixLeftAndRightFiltersNotPushableToLucene
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+| WHERE ABS(other2) > 5000 AND (extra1 == "qux" OR extra1 == "foo2") AND other1 like ("*ta", "*ron")
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int, name_str, extra1, other1, other2
+| LIMIT 20
+;
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+4 | David | qux | zeta | 6000
+14 | Nina | foo2 | omicron | 15000
+;
+
+
+lookupJoinWithMixJoinAndNonJoinColumnsNotPushable
+required_capability: join_lookup_v12
+required_capability: lookup_join_on_multiple_fields
+
+FROM multi_column_joinable
+| LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+| WHERE ABS(other2) > id_int + 5000
+| KEEP id_int, name_str, extra1, other1, other2
+| SORT id_int, name_str, extra1, other1, other2
+| LIMIT 20
+;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+warning:Line 3:23: evaluation of [id_int + 5000] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:23: java.lang.IllegalArgumentException: single-value function encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+4 | David | qux | zeta | 6000
+5 | Eve | quux | eta | 7000
+5 | Eve | quux | theta | 8000
+6 | null | corge | iota | 9000
+7 | Grace | grault | kappa | 10000
+8 | Hank | garply | lambda | 11000
+12 | Liam | xyzzy | nu | 13000
+13 | Mia | thud | xi | 14000
+14 | Nina | foo2 | omicron | 15000
+;
+
+
+lookupJoinWithMixJoinAndNonJoinColumnsPushable
+ required_capability: join_lookup_v12
+ required_capability: lookup_join_on_multiple_fields
+
+ FROM multi_column_joinable
+ | LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+ | WHERE other2 > id_int + 5000
+ | KEEP id_int, name_str, extra1, other1, other2
+ | SORT id_int, name_str, extra1, other1, other2
+ | LIMIT 20
+ ;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+warning:Line 3:18: evaluation of [id_int + 5000] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:18: java.lang.IllegalArgumentException: single-value function encountered multi-value
+
+ id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+ 4 | David | qux | zeta | 6000
+ 5 | Eve | quux | eta | 7000
+ 5 | Eve | quux | theta | 8000
+ 6 | null | corge | iota | 9000
+ 7 | Grace | grault | kappa | 10000
+ 8 | Hank | garply | lambda | 11000
+ 12 | Liam | xyzzy | nu | 13000
+ 13 | Mia | thud | xi | 14000
+ 14 | Nina | foo2 | omicron | 15000
+ ;
+
+lookupJoinWithMixPushableAndUnpushableFilters
+ required_capability: join_lookup_v12
+ required_capability: lookup_join_on_multiple_fields
+
+ FROM multi_column_joinable
+ | LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+ | WHERE other2 > id_int + 5000 AND (extra1 == "qux" OR extra1 == "zyx") AND other1 like "*ta" AND ABS(other2) > 5500
+ | KEEP id_int, name_str, extra1, other1, other2
+ | SORT id_int, name_str, extra1, other1, other2
+ | LIMIT 20
+ ;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+warning:Line 3:18: evaluation of [id_int + 5000] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:18: java.lang.IllegalArgumentException: single-value function encountered multi-value
+
+ id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+ 4 | David | qux | zeta | 6000
+ ;
+
+ lookupJoinWithJoinAttrFilter
+ required_capability: join_lookup_v12
+ required_capability: lookup_join_on_multiple_fields
+
+ FROM multi_column_joinable
+ | LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+ | WHERE id_int > 7
+ | KEEP id_int, name_str, extra1, other1, other2
+ | SORT id_int, name_str, extra1, other1, other2
+ | LIMIT 20
+ ;
+
+warning:Line 3:9: evaluation of [id_int > 7] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 3:9: java.lang.IllegalArgumentException: single-value function encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+8 | Hank | garply | lambda | 11000
+9 | null | waldo | null | null
+10 | null | fred | null | null
+12 | Liam | xyzzy | nu | 13000
+13 | Mia | thud | xi | 14000
+14 | Nina | foo2 | omicron | 15000
+15 | null | bar2 | null | null
+;
+
+
+lookupJoinWithExpressionOfOtherFields
+ required_capability: join_lookup_v12
+ required_capability: lookup_join_on_multiple_fields
+
+ FROM multi_column_joinable
+ | LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool
+ | WHERE ABS(other2) > LENGTH(other1)*1000 + 2000
+ | KEEP id_int, name_str, extra1, other1, other2
+ | SORT id_int, name_str, extra1, other1, other2
+ | LIMIT 20
+ ;
+
+warning:Line 2:3: evaluation of [LOOKUP JOIN multi_column_joinable_lookup ON id_int, is_active_bool] failed, treating result as null. Only first 20 failures recorded.
+warning:Line 2:3: java.lang.IllegalArgumentException: LOOKUP JOIN encountered multi-value
+
+id_int:integer | name_str:keyword | extra1:keyword | other1:keyword | other2:integer
+5 | Eve | quux | eta | 7000
+5 | Eve | quux | theta | 8000
+6 | null | corge | iota | 9000
+7 | Grace | grault | kappa | 10000
+8 | Hank | garply | lambda | 11000
+12 | Liam | xyzzy | nu | 13000
+13 | Mia | thud | xi | 14000
+14 | Nina | foo2 | omicron | 15000
+;
diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java
index dc8a14abf1e12..b741985a0a3ca 100644
--- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java
+++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java
@@ -22,7 +22,6 @@
import org.elasticsearch.compute.data.Block;
import org.elasticsearch.compute.data.BlockFactory;
import org.elasticsearch.compute.data.LongBlock;
-import org.elasticsearch.compute.data.LongVector;
import org.elasticsearch.compute.lucene.DataPartitioning;
import org.elasticsearch.compute.lucene.LuceneOperator;
import org.elasticsearch.compute.lucene.LuceneSliceQueue;
@@ -36,6 +35,7 @@
import org.elasticsearch.compute.test.BlockTestUtils;
import org.elasticsearch.compute.test.TestDriverFactory;
import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.index.IndexMode;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexVersion;
@@ -54,12 +54,20 @@
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.xpack.core.async.AsyncExecutionId;
import org.elasticsearch.xpack.esql.core.expression.Alias;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
import org.elasticsearch.xpack.esql.core.tree.Source;
import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.core.type.EsField;
import org.elasticsearch.xpack.esql.enrich.LookupFromIndexOperator;
import org.elasticsearch.xpack.esql.enrich.MatchConfig;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThan;
+import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
+import org.elasticsearch.xpack.esql.plan.logical.Filter;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
import org.elasticsearch.xpack.esql.planner.EsPhysicalOperationProviders;
import org.elasticsearch.xpack.esql.planner.PhysicalSettings;
import org.elasticsearch.xpack.esql.planner.PlannerUtils;
@@ -75,6 +83,7 @@
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Predicate;
import static org.elasticsearch.test.ListMatcher.matchesList;
import static org.elasticsearch.test.MapMatcher.assertMap;
@@ -83,13 +92,14 @@
public class LookupFromIndexIT extends AbstractEsqlIntegTestCase {
public void testKeywordKey() throws IOException {
- runLookup(List.of(DataType.KEYWORD), new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "cc", "dd" } }));
+ runLookup(List.of(DataType.KEYWORD), new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "cc", "dd" } }), null);
}
public void testJoinOnTwoKeys() throws IOException {
runLookup(
List.of(DataType.KEYWORD, DataType.LONG),
- new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "cc", "dd" }, new Long[] { 12L, 33L, 1L, 42L } })
+ new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "cc", "dd" }, new Long[] { 12L, 33L, 1L, 42L } }),
+ null
);
}
@@ -101,7 +111,8 @@ public void testJoinOnThreeKeys() throws IOException {
new String[] { "aa", "bb", "cc", "dd" },
new Long[] { 12L, 33L, 1L, 42L },
new String[] { "one", "two", "three", "four" }, }
- )
+ ),
+ null
);
}
@@ -114,25 +125,35 @@ public void testJoinOnFourKeys() throws IOException {
new Long[] { 12L, 33L, 1L, 42L },
new String[] { "one", "two", "three", "four" },
new Integer[] { 1, 2, 3, 4 }, }
- )
+ ),
+ buildGreaterThanFilter(1L)
);
}
public void testLongKey() throws IOException {
- runLookup(List.of(DataType.LONG), new UsingSingleLookupTable(new Object[][] { new Long[] { 12L, 33L, 1L } }));
+ runLookup(
+ List.of(DataType.LONG),
+ new UsingSingleLookupTable(new Object[][] { new Long[] { 12L, 33L, 1L } }),
+ buildGreaterThanFilter(0L)
+ );
}
/**
* LOOKUP multiple results match.
*/
public void testLookupIndexMultiResults() throws IOException {
- runLookup(List.of(DataType.KEYWORD), new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "bb", "dd" } }));
+ runLookup(
+ List.of(DataType.KEYWORD),
+ new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "bb", "dd" } }),
+ buildGreaterThanFilter(-1L)
+ );
}
public void testJoinOnTwoKeysMultiResults() throws IOException {
runLookup(
List.of(DataType.KEYWORD, DataType.LONG),
- new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "bb", "dd" }, new Long[] { 12L, 1L, 1L, 42L } })
+ new UsingSingleLookupTable(new Object[][] { new String[] { "aa", "bb", "bb", "dd" }, new Long[] { 12L, 1L, 1L, 42L } }),
+ null
);
}
@@ -144,12 +165,13 @@ public void testJoinOnThreeKeysMultiResults() throws IOException {
new String[] { "aa", "bb", "bb", "dd" },
new Long[] { 12L, 1L, 1L, 42L },
new String[] { "one", "two", "two", "four" } }
- )
+ ),
+ null
);
}
interface PopulateIndices {
- void populate(int docCount, List expected) throws IOException;
+ void populate(int docCount, List expected, Predicate filter) throws IOException;
}
class UsingSingleLookupTable implements PopulateIndices {
@@ -171,7 +193,7 @@ class UsingSingleLookupTable implements PopulateIndices {
}
@Override
- public void populate(int docCount, List expected) {
+ public void populate(int docCount, List expected, Predicate filter) {
List docs = new ArrayList<>();
int numFields = lookupData.length;
int numRows = lookupData[0].length;
@@ -190,8 +212,13 @@ public void populate(int docCount, List expected) {
} else {
keyString = String.join(",", key.stream().map(String::valueOf).toArray(String[]::new));
}
- for (Integer match : matches.get(key)) {
- expected.add(keyString + ":" + match);
+ List filteredMatches = matches.get(key).stream().filter(filter).toList();
+ if (filteredMatches.isEmpty()) {
+ expected.add(keyString + ":null");
+ } else {
+ for (Integer match : filteredMatches) {
+ expected.add(keyString + ":" + match);
+ }
}
}
for (int i = 0; i < numRows; i++) {
@@ -207,7 +234,19 @@ public void populate(int docCount, List expected) {
}
}
- private void runLookup(List keyTypes, PopulateIndices populateIndices) throws IOException {
+ private PhysicalPlan buildGreaterThanFilter(long value) {
+ FieldAttribute filterAttribute = new FieldAttribute(
+ Source.EMPTY,
+ "l",
+ new EsField("l", DataType.LONG, Collections.emptyMap(), true, EsField.TimeSeriesFieldType.NONE)
+ );
+ Expression greaterThan = new GreaterThan(Source.EMPTY, filterAttribute, new Literal(Source.EMPTY, value, DataType.LONG));
+ EsRelation esRelation = new EsRelation(Source.EMPTY, "test", IndexMode.LOOKUP, Map.of(), List.of());
+ Filter filter = new Filter(Source.EMPTY, esRelation, greaterThan);
+ return new FragmentExec(filter);
+ }
+
+ private void runLookup(List keyTypes, PopulateIndices populateIndices, PhysicalPlan filters) throws IOException {
String[] fieldMappers = new String[keyTypes.size() * 2];
for (int i = 0; i < keyTypes.size(); i++) {
fieldMappers[2 * i] = "key" + i;
@@ -236,9 +275,23 @@ private void runLookup(List keyTypes, PopulateIndices populateIndices)
client().admin().cluster().prepareHealth(TEST_REQUEST_TIMEOUT).setWaitForGreenStatus().get();
+ Predicate filterPredicate = l -> true;
+ if (filters instanceof FragmentExec fragmentExec) {
+ if (fragmentExec.fragment() instanceof Filter filter
+ && filter.condition() instanceof GreaterThan gt
+ && gt.left() instanceof FieldAttribute fa
+ && fa.name().equals("l")
+ && gt.right() instanceof Literal lit) {
+ long value = ((Number) lit.value()).longValue();
+ filterPredicate = l -> l > value;
+ } else {
+ fail("Unsupported filter type in test baseline generation: " + filters);
+ }
+ }
+
int docCount = between(10, 1000);
List expected = new ArrayList<>(docCount);
- populateIndices.populate(docCount, expected);
+ populateIndices.populate(docCount, expected, filterPredicate);
/*
* Find the data node hosting the only shard of the source index.
@@ -331,7 +384,9 @@ private void runLookup(List keyTypes, PopulateIndices populateIndices)
"lookup",
"lookup",
List.of(new Alias(Source.EMPTY, "l", new ReferenceAttribute(Source.EMPTY, "l", DataType.LONG))),
- Source.EMPTY
+ Source.EMPTY,
+ filters,
+ null
);
DriverContext driverContext = driverContext();
try (
@@ -345,7 +400,7 @@ private void runLookup(List keyTypes, PopulateIndices populateIndices)
for (int i = 0; i < keyTypes.size(); i++) {
keyBlocks.add(page.getBlock(i + 1));
}
- LongVector loadedBlock = page.getBlock(keyTypes.size() + 1).asVector();
+ LongBlock loadedBlock = page.getBlock(keyTypes.size() + 1);
for (int p = 0; p < page.getPositionCount(); p++) {
StringBuilder result = new StringBuilder();
for (int j = 0; j < keyBlocks.size(); j++) {
@@ -361,7 +416,11 @@ private void runLookup(List keyTypes, PopulateIndices populateIndices)
}
}
- result.append(":" + loadedBlock.getLong(p));
+ if (loadedBlock.isNull(p)) {
+ result.append(":null");
+ } else {
+ result.append(":" + loadedBlock.getLong(loadedBlock.getFirstValueIndex(p)));
+ }
results.add(result.toString());
}
} finally {
diff --git a/x-pack/plugin/esql/src/main/antlr/parser/Expression.g4 b/x-pack/plugin/esql/src/main/antlr/parser/Expression.g4
index 135a104b89139..9de7b5e8084bb 100644
--- a/x-pack/plugin/esql/src/main/antlr/parser/Expression.g4
+++ b/x-pack/plugin/esql/src/main/antlr/parser/Expression.g4
@@ -30,7 +30,11 @@ matchBooleanExpression
valueExpression
: operatorExpression #valueExpressionDefault
- | left=operatorExpression comparisonOperator right=operatorExpression #comparison
+ | comparisonExpression #comparison
+ ;
+
+comparisonExpression
+ : left=operatorExpression comparisonOperator right=operatorExpression
;
operatorExpression
diff --git a/x-pack/plugin/esql/src/main/antlr/parser/Join.g4 b/x-pack/plugin/esql/src/main/antlr/parser/Join.g4
index facbd38fe3908..b6f5f4c26467e 100644
--- a/x-pack/plugin/esql/src/main/antlr/parser/Join.g4
+++ b/x-pack/plugin/esql/src/main/antlr/parser/Join.g4
@@ -18,11 +18,8 @@ joinTarget
;
joinCondition
- : ON joinPredicate (COMMA joinPredicate)*
- ;
-
-joinPredicate
- : valueExpression
+ : ON qualifiedName (COMMA qualifiedName)* #fieldBasedLookupJoin
+ | ON comparisonExpression (AND comparisonExpression)* #expressionBasedLookupJoin
;
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
index e58fa2eeb5856..9e0b1cf91790e 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java
@@ -1399,7 +1399,12 @@ public enum Cap {
/**
* Allow qualifiers in attribute names.
*/
- NAME_QUALIFIERS(Build.current().isSnapshot());
+ NAME_QUALIFIERS(Build.current().isSnapshot()),
+
+ /**
+ * Allow lookup join on boolean expressions
+ */
+ LOOKUP_JOIN_ON_BOOLEAN_EXPRESSION;
private final boolean enabled;
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java
index 50a061ab0d6ef..3611a3837ce7b 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java
@@ -83,8 +83,10 @@
import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToUnsignedLong;
import org.elasticsearch.xpack.esql.expression.function.scalar.nulls.Coalesce;
import org.elasticsearch.xpack.esql.expression.function.vector.VectorFunction;
+import org.elasticsearch.xpack.esql.expression.predicate.Predicates;
import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.DateTimeArithmeticOperation;
import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.EsqlArithmeticOperation;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison;
import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In;
import org.elasticsearch.xpack.esql.index.EsIndex;
import org.elasticsearch.xpack.esql.index.IndexResolution;
@@ -441,7 +443,14 @@ protected LogicalPlan rule(Lookup lookup, AnalyzerContext context) {
// postpone the resolution for ResolveRefs
}
- return new Lookup(source, lookup.child(), tableNameExpression, lookup.matchFields(), localRelation);
+ return new Lookup(
+ source,
+ lookup.child(),
+ tableNameExpression,
+ lookup.matchFields(),
+ localRelation,
+ lookup.getJoinOnConditions()
+ );
}
private LocalRelation tableMapAsRelation(Source source, Map mapTable) {
@@ -707,15 +716,30 @@ private LogicalPlan resolveLookup(Lookup l, List childrenOutput) {
matchFields.add(matchFieldChildReference);
}
if (modified) {
- return new Lookup(l.source(), l.child(), l.tableName(), matchFields, l.localRelation());
+ return new Lookup(l.source(), l.child(), l.tableName(), matchFields, l.localRelation(), l.getJoinOnConditions());
}
return l;
}
+ private List resolveJoinFilters(List filters, List leftOutput, List rightOutput) {
+ if (filters.isEmpty()) {
+ return emptyList();
+ }
+ List childrenOutput = new ArrayList<>(leftOutput);
+ childrenOutput.addAll(rightOutput);
+
+ List resolvedFilters = new ArrayList<>(filters.size());
+ for (Expression filter : filters) {
+ resolvedFilters.add(filter.transformUp(UnresolvedAttribute.class, ua -> maybeResolveAttribute(ua, childrenOutput)));
+ }
+ return resolvedFilters;
+ }
+
private Join resolveLookupJoin(LookupJoin join) {
JoinConfig config = join.config();
// for now, support only (LEFT) USING clauses
JoinType type = config.type();
+
// rewrite the join into an equi-join between the field with the same name between left and right
if (type instanceof UsingJoinType using) {
List cols = using.columns();
@@ -733,22 +757,48 @@ private Join resolveLookupJoin(LookupJoin join) {
name,
"Only LEFT join is supported with USING"
);
- return join.withConfig(new JoinConfig(type, singletonList(errorAttribute), emptyList(), emptyList()));
+ return join.withConfig(new JoinConfig(type, singletonList(errorAttribute), emptyList(), emptyList(), null));
+ }
+ List leftKeys = new ArrayList<>();
+ List rightKeys = new ArrayList<>();
+ List resolvedFilters = new ArrayList<>();
+ List matchKeys;
+ if (join.config().joinOnConditions() != null) {
+ resolvedFilters = resolveJoinFilters(
+ Predicates.splitAnd(join.config().joinOnConditions()),
+ join.left().output(),
+ join.right().output()
+ );
+ // build leftKeys and rightKeys using the left side of the resolvedFilters.
+ for (Expression expression : resolvedFilters) {
+ if (expression instanceof EsqlBinaryComparison binaryComparison) {
+ leftKeys.add((Attribute) binaryComparison.left());
+ rightKeys.add((Attribute) binaryComparison.right());
+ } else {
+ throw new EsqlIllegalArgumentException("Unsupported join filter expression: " + expression);
+ }
+ }
+ Set matchKeysSet = new HashSet<>(leftKeys);
+ matchKeysSet.addAll(rightKeys);
+ matchKeys = new ArrayList<>(matchKeysSet);
+ } else {
+ // resolve the using columns against the left and the right side then assemble the new join config
+ leftKeys = resolveUsingColumns(cols, join.left().output(), "left");
+ rightKeys = resolveUsingColumns(cols, join.right().output(), "right");
+ matchKeys = leftKeys;
}
- // resolve the using columns against the left and the right side then assemble the new join config
- List leftKeys = resolveUsingColumns(cols, join.left().output(), "left");
- List rightKeys = resolveUsingColumns(cols, join.right().output(), "right");
- config = new JoinConfig(coreJoin, leftKeys, leftKeys, rightKeys);
- join = new LookupJoin(join.source(), join.left(), join.right(), config, join.isRemote());
+ config = new JoinConfig(coreJoin, matchKeys, leftKeys, rightKeys, Predicates.combineAnd(resolvedFilters));
+ return new LookupJoin(join.source(), join.left(), join.right(), config, join.isRemote());
} else if (type != JoinTypes.LEFT) {
// everything else is unsupported for now
// LEFT can only happen by being mapped from a USING above. So we need to exclude this as well because this rule can be run
// more than once.
UnresolvedAttribute errorAttribute = new UnresolvedAttribute(join.source(), "unsupported", "Unsupported join type");
// add error message
- return join.withConfig(new JoinConfig(type, singletonList(errorAttribute), emptyList(), emptyList()));
+ return join.withConfig(new JoinConfig(type, singletonList(errorAttribute), emptyList(), emptyList(), null));
}
+
return join;
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java
index b52c6472f962a..c834b538c5e86 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java
@@ -558,6 +558,10 @@ abstract static class TransportRequest extends AbstractTransportRequest implemen
this.source = source;
}
+ public Page getInputPage() {
+ return inputPage;
+ }
+
@Override
public final String[] indices() {
return new String[] { indexPattern };
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/BinaryComparisonQueryList.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/BinaryComparisonQueryList.java
new file mode 100644
index 0000000000000..4667c25204f03
--- /dev/null
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/BinaryComparisonQueryList.java
@@ -0,0 +1,88 @@
+/*
+ * 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.enrich;
+
+import org.apache.lucene.search.Query;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.compute.data.Block;
+import org.elasticsearch.compute.operator.Warnings;
+import org.elasticsearch.compute.operator.lookup.QueryList;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.query.SearchExecutionContext;
+import org.elasticsearch.search.internal.AliasFilter;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates;
+import org.elasticsearch.xpack.esql.planner.TranslatorHandler;
+import org.elasticsearch.xpack.esql.plugin.EsqlFlags;
+import org.elasticsearch.xpack.esql.stats.SearchContextStats;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.List;
+import java.util.function.IntFunction;
+
+public class BinaryComparisonQueryList extends QueryList {
+ private final EsqlBinaryComparison binaryComparison;
+ private final IntFunction blockValueReader;
+ private final SearchExecutionContext searchExecutionContext;
+ private final LucenePushdownPredicates lucenePushdownPredicates;
+
+ public BinaryComparisonQueryList(
+ MappedFieldType field,
+ SearchExecutionContext searchExecutionContext,
+ Block block,
+ EsqlBinaryComparison binaryComparison,
+ ClusterService clusterService,
+ AliasFilter aliasFilter,
+ Warnings warnings
+ ) {
+ super(
+ field,
+ searchExecutionContext,
+ aliasFilter,
+ block,
+ new OnlySingleValueParams(warnings, "LOOKUP JOIN encountered multi-value")
+ );
+ // swap left and right if the field is on the right
+ // We get a filter in the form left_expr >= right_expr
+ // here we will swap it to right_expr <= left_expr
+ // and later in doGetQuery we will replace left_expr with the value from the block
+ this.binaryComparison = (EsqlBinaryComparison) binaryComparison.swapLeftAndRight();
+ this.blockValueReader = QueryList.createBlockValueReader(block);
+ this.searchExecutionContext = searchExecutionContext;
+ lucenePushdownPredicates = LucenePushdownPredicates.from(
+ SearchContextStats.from(List.of(searchExecutionContext)),
+ new EsqlFlags(clusterService.getClusterSettings())
+ );
+ }
+
+ @Override
+ public QueryList onlySingleValues(Warnings warnings, String multiValueWarningMessage) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Query doGetQuery(int position, int firstValueIndex, int valueCount) {
+ Object value = blockValueReader.apply(firstValueIndex);
+ // create a new comparison with the value from the block as a literal
+ EsqlBinaryComparison comparison = binaryComparison.getFunctionType()
+ .buildNewInstance(
+ binaryComparison.source(),
+ binaryComparison.left(),
+ new Literal(binaryComparison.right().source(), value, binaryComparison.right().dataType())
+ );
+ try {
+ return comparison.asQuery(lucenePushdownPredicates, TranslatorHandler.TRANSLATOR_HANDLER)
+ .toQueryBuilder()
+ .toQuery(searchExecutionContext);
+ } catch (IOException e) {
+ throw new UncheckedIOException("Error while building query for join on filter:", e);
+ }
+ }
+}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/ExpressionQueryList.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/ExpressionQueryList.java
new file mode 100644
index 0000000000000..747d44717d81e
--- /dev/null
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/ExpressionQueryList.java
@@ -0,0 +1,230 @@
+/*
+ * 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.enrich;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.elasticsearch.cluster.service.ClusterService;
+import org.elasticsearch.compute.data.Block;
+import org.elasticsearch.compute.operator.Warnings;
+import org.elasticsearch.compute.operator.lookup.LookupEnrichQueryGenerator;
+import org.elasticsearch.compute.operator.lookup.QueryList;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.query.SearchExecutionContext;
+import org.elasticsearch.search.internal.AliasFilter;
+import org.elasticsearch.xpack.esql.capabilities.TranslationAware;
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.expression.predicate.Predicates;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison;
+import org.elasticsearch.xpack.esql.optimizer.rules.physical.local.LucenePushdownPredicates;
+import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
+import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.plan.physical.UnaryExec;
+import org.elasticsearch.xpack.esql.plugin.EsqlFlags;
+import org.elasticsearch.xpack.esql.stats.SearchContextStats;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.elasticsearch.xpack.esql.planner.TranslatorHandler.TRANSLATOR_HANDLER;
+
+/**
+ * A {@link LookupEnrichQueryGenerator} that combines one or more {@link QueryList}s into a single query.
+ * Each query in the resulting query will be a conjunction of all queries from the input lists at the same position.
+ * In addition, we support an optional pre-join filter that will be applied to all queries if it is pushable.
+ * If the pre-join filter cannot be pushed down to Lucene, it will be ignored.
+ */
+public class ExpressionQueryList implements LookupEnrichQueryGenerator {
+ private static final Logger logger = LogManager.getLogger(ExpressionQueryList.class);
+ private final List queryLists;
+ private final List preJoinFilters = new ArrayList<>();
+ private final SearchExecutionContext context;
+ boolean isExpressionJoin = false;
+ private final AliasFilter aliasFilter;
+
+ public ExpressionQueryList(
+ List queryLists,
+ SearchExecutionContext context,
+ PhysicalPlan rightPreJoinPlan,
+ ClusterService clusterService,
+ LookupFromIndexService.TransportRequest request,
+ AliasFilter aliasFilter,
+ Warnings warnings
+ ) {
+ if (queryLists.size() < 2 && (rightPreJoinPlan instanceof FilterExec == false) && request.getJoinOnConditions() == null) {
+ throw new IllegalArgumentException("ExpressionQueryList must have at least two QueryLists or a pre-join filter");
+ }
+ this.queryLists = queryLists;
+ this.context = context;
+ this.aliasFilter = aliasFilter;
+ buildJoinOnConditions(request, clusterService, warnings);
+ buildPreJoinFilter(rightPreJoinPlan, clusterService);
+
+ }
+
+ private void buildJoinOnConditions(LookupFromIndexService.TransportRequest request, ClusterService clusterService, Warnings warnings) {
+ // we support 2 modes of operation:
+ // Join on fields
+ // Join on AND of binary comparisons
+ Expression filter = request.getJoinOnConditions();
+ if (filter == null) {
+ // the join on conditions are already populated via the queryLists
+ // there is nothing to do here
+ return;
+ } else {
+ // clear the join on conditions in the query lists
+ // the join on condition needs to come from the expression
+ queryLists.clear();
+ }
+ List expressions = Predicates.splitAnd(filter);
+ for (Expression expr : expressions) {
+ if (expr instanceof EsqlBinaryComparison binaryComparison) {
+ // the left side comes from the page that was sent to the lookup node
+ // the right side is the field from the lookup index
+ // check if the left side is in the request.getMatchFields()
+ // if it is its corresponding page is the corresponding number in request.inputPage
+ Expression left = binaryComparison.left();
+ if (left instanceof Attribute leftAttribute) {
+ boolean matched = false;
+ for (int i = 0; i < request.getMatchFields().size(); i++) {
+ if (request.getMatchFields().get(i).fieldName().string().equals(leftAttribute.name())) {
+ Block block = request.getInputPage().getBlock(i);
+ Expression right = binaryComparison.right();
+ if (right instanceof Attribute rightAttribute) {
+ MappedFieldType fieldType = context.getFieldType(rightAttribute.name());
+ if (fieldType != null) {
+ isExpressionJoin = true;
+ queryLists.add(
+ new BinaryComparisonQueryList(
+ fieldType,
+ context,
+ block,
+ binaryComparison,
+ clusterService,
+ aliasFilter,
+ warnings
+ )
+ );
+ matched = true;
+ break;
+ } else {
+ throw new IllegalArgumentException(
+ "Could not find field [" + rightAttribute.name() + "] in the lookup join index"
+ );
+ }
+ } else {
+ throw new IllegalArgumentException(
+ "Only field from the right dataset are supported on the right of the join on condition but got: " + expr
+ );
+ }
+ }
+ }
+ if (matched == false) {
+ throw new IllegalArgumentException(
+ "Could not find field [" + leftAttribute.name() + "] in the left side of the lookup join"
+ );
+ }
+ } else {
+ throw new IllegalArgumentException(
+ "Only field from the left dataset are supported on the left of the join on condition but got: " + expr
+ );
+ }
+ } else {
+ // we only support binary comparisons in the join on conditions
+ throw new IllegalArgumentException("Only binary comparisons are supported in join ON conditions, but got: " + expr);
+ }
+ }
+ }
+
+ private void addToPreJoinFilters(org.elasticsearch.index.query.QueryBuilder query) {
+ try {
+ if (query != null) {
+ preJoinFilters.add(query.toQuery(context));
+ }
+ } catch (IOException e) {
+ // as we treat the filter as optional an error in its application will be ignored
+ logger.error(() -> "Failed to translate optional pre-join filter: [" + query + "]", e);
+ }
+ }
+
+ private void buildPreJoinFilter(PhysicalPlan rightPreJoinPlan, ClusterService clusterService) {
+ if (rightPreJoinPlan instanceof EsQueryExec esQueryExec) {
+ // this does not happen right now, as we only do local mapping on the lookup node
+ // so we have EsSourceExec, not esQueryExec
+ if (esQueryExec.query() != null) {
+ addToPreJoinFilters(esQueryExec.query());
+ }
+ } else if (rightPreJoinPlan instanceof FilterExec filterExec) {
+ List candidateRightHandFilters = Predicates.splitAnd(filterExec.condition());
+ LucenePushdownPredicates lucenePushdownPredicates = LucenePushdownPredicates.from(
+ SearchContextStats.from(List.of(context)),
+ new EsqlFlags(clusterService.getClusterSettings())
+ );
+ for (Expression filter : candidateRightHandFilters) {
+ if (filter instanceof TranslationAware translationAware) {
+ if (TranslationAware.Translatable.YES.equals(translationAware.translatable(lucenePushdownPredicates))) {
+ addToPreJoinFilters(translationAware.asQuery(lucenePushdownPredicates, TRANSLATOR_HANDLER).toQueryBuilder());
+ }
+ }
+ // If the filter is not translatable we will not apply it for now
+ // as performance testing showed no performance improvement.
+ // We can revisit this in the future if needed, once we have more optimized workflow in place.
+ // The filter is optional, so it is OK to ignore it if it cannot be translated.
+ }
+ // call recursively to find other filters that might be present
+ // either in another FilterExec or in an EsQueryExec
+ buildPreJoinFilter(filterExec.child(), clusterService);
+ } else if (rightPreJoinPlan instanceof UnaryExec unaryExec) {
+ // there can be other nodes in the plan such as FieldExtractExec in the future
+ buildPreJoinFilter(unaryExec.child(), clusterService);
+ }
+ // else we do nothing, as the filters are optional and we don't want to fail the query if there are any errors
+ // this also covers the case of rightPreJoinPlan being null
+ }
+
+ @Override
+ public Query getQuery(int position) {
+ BooleanQuery.Builder builder = new BooleanQuery.Builder();
+ for (QueryList queryList : queryLists) {
+ Query q = queryList.getQuery(position);
+ if (q == null) {
+ // if any of the matchFields are null, it means there is no match for this position
+ // A AND NULL is always NULL, so we can skip this position
+ return null;
+ }
+ builder.add(q, BooleanClause.Occur.FILTER);
+ }
+ // also attach the pre-join filter if it exists
+ for (Query preJoinFilter : preJoinFilters) {
+ builder.add(preJoinFilter, BooleanClause.Occur.FILTER);
+ }
+ return builder.build();
+ }
+
+ @Override
+ public int getPositionCount() {
+ int positionCount = queryLists.get(0).getPositionCount();
+ for (QueryList queryList : queryLists) {
+ if (queryList.getPositionCount() != positionCount) {
+ throw new IllegalArgumentException(
+ "All QueryLists must have the same position count, expected: "
+ + positionCount
+ + ", but got: "
+ + queryList.getPositionCount()
+ );
+ }
+ }
+ return positionCount;
+ }
+}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperator.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperator.java
index 641f3420bbd78..2bb86a7ebf98d 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperator.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperator.java
@@ -24,8 +24,10 @@
import org.elasticsearch.core.Releasables;
import org.elasticsearch.tasks.CancellableTask;
import org.elasticsearch.xcontent.XContentBuilder;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
import org.elasticsearch.xpack.esql.core.tree.Source;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
import java.io.IOException;
import java.util.ArrayList;
@@ -47,7 +49,9 @@ public record Factory(
String lookupIndexPattern,
String lookupIndex,
List loadFields,
- Source source
+ Source source,
+ PhysicalPlan rightPreJoinPlan,
+ Expression joinOnConditions
) implements OperatorFactory {
@Override
public String describe() {
@@ -61,6 +65,8 @@ public String describe() {
.append(" inputChannel=")
.append(matchField.channel());
}
+ stringBuilder.append(" right_pre_join_plan=").append(rightPreJoinPlan == null ? "null" : rightPreJoinPlan.toString());
+ stringBuilder.append(" join_on_expression=").append(joinOnConditions == null ? "null" : joinOnConditions.toString());
stringBuilder.append("]");
return stringBuilder.toString();
}
@@ -77,7 +83,9 @@ public Operator get(DriverContext driverContext) {
lookupIndexPattern,
lookupIndex,
loadFields,
- source
+ source,
+ rightPreJoinPlan,
+ joinOnConditions
);
}
}
@@ -90,7 +98,9 @@ public Operator get(DriverContext driverContext) {
private final List loadFields;
private final Source source;
private long totalRows = 0L;
- private List matchFields;
+ private final List matchFields;
+ private final PhysicalPlan rightPreJoinPlan;
+ private final Expression joinOnConditions;
/**
* Total number of pages emitted by this {@link Operator}.
*/
@@ -114,7 +124,9 @@ public LookupFromIndexOperator(
String lookupIndexPattern,
String lookupIndex,
List loadFields,
- Source source
+ Source source,
+ PhysicalPlan rightPreJoinPlan,
+ Expression joinOnConditions
) {
super(driverContext, lookupService.getThreadContext(), maxOutstandingRequests);
this.matchFields = matchFields;
@@ -125,6 +137,8 @@ public LookupFromIndexOperator(
this.lookupIndex = lookupIndex;
this.loadFields = loadFields;
this.source = source;
+ this.rightPreJoinPlan = rightPreJoinPlan;
+ this.joinOnConditions = joinOnConditions;
}
@Override
@@ -151,7 +165,9 @@ protected void performAsync(Page inputPage, ActionListener listener
newMatchFields,
new Page(inputBlockArray),
loadFields,
- source
+ source,
+ rightPreJoinPlan,
+ joinOnConditions
);
lookupService.lookupAsync(
request,
@@ -211,6 +227,9 @@ public String toString() {
.append(" inputChannel=")
.append(matchField.channel());
}
+
+ stringBuilder.append(" right_pre_join_plan=").append(rightPreJoinPlan == null ? "null" : rightPreJoinPlan.toString());
+ stringBuilder.append(" join_on_expression=").append(joinOnConditions == null ? "null" : joinOnConditions.toString());
stringBuilder.append("]");
return stringBuilder.toString();
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexService.java
index 4cfa7adaede81..dfe5cd5cadc5d 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexService.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexService.java
@@ -7,6 +7,8 @@
package org.elasticsearch.xpack.esql.enrich;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
import org.elasticsearch.TransportVersion;
import org.elasticsearch.TransportVersions;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
@@ -21,7 +23,6 @@
import org.elasticsearch.compute.data.BlockStreamInput;
import org.elasticsearch.compute.data.Page;
import org.elasticsearch.compute.operator.Warnings;
-import org.elasticsearch.compute.operator.lookup.ExpressionQueryList;
import org.elasticsearch.compute.operator.lookup.LookupEnrichQueryGenerator;
import org.elasticsearch.compute.operator.lookup.QueryList;
import org.elasticsearch.core.Releasables;
@@ -33,12 +34,17 @@
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;
import org.elasticsearch.xpack.esql.action.EsqlQueryAction;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
import org.elasticsearch.xpack.esql.core.tree.Source;
import org.elasticsearch.xpack.esql.core.type.DataType;
import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput;
import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput;
+import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
+import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
+import org.elasticsearch.xpack.esql.planner.mapper.LocalMapper;
import java.io.IOException;
import java.util.ArrayList;
@@ -53,6 +59,7 @@
*/
public class LookupFromIndexService extends AbstractLookupService {
public static final String LOOKUP_ACTION_NAME = EsqlQueryAction.NAME + "/lookup_from_index";
+ private static final Logger logger = LogManager.getLogger(LookupFromIndexService.class);
public LookupFromIndexService(
ClusterService clusterService,
@@ -89,7 +96,9 @@ protected TransportRequest transportRequest(LookupFromIndexService.Request reque
null,
request.extractFields,
request.matchFields,
- request.source
+ request.source,
+ request.rightPreJoinPlan,
+ request.joinOnConditions
);
}
@@ -113,10 +122,32 @@ protected LookupEnrichQueryGenerator queryList(
).onlySingleValues(warnings, "LOOKUP JOIN encountered multi-value");
queryLists.add(q);
}
- if (queryLists.size() == 1) {
+
+ PhysicalPlan physicalPlan = request.rightPreJoinPlan;
+ physicalPlan = localLookupNodePlanning(physicalPlan);
+ if (queryLists.size() == 1 && physicalPlan instanceof FilterExec == false && request.joinOnConditions == null) {
return queryLists.getFirst();
}
- return new ExpressionQueryList(queryLists);
+ return new ExpressionQueryList(queryLists, context, physicalPlan, clusterService, request, aliasFilter, warnings);
+
+ }
+
+ /**
+ * This function will perform any planning needed on the local node
+ * For now, we will just do mapping of the logical plan to physical plan
+ * In the future we can also do local physical and logical optimizations
+ */
+ private static PhysicalPlan localLookupNodePlanning(PhysicalPlan physicalPlan) {
+ if (physicalPlan instanceof FragmentExec fragmentExec) {
+ try {
+ LocalMapper localMapper = new LocalMapper();
+ return localMapper.map(fragmentExec.fragment());
+ } catch (Exception e) {
+ logger.error(() -> "Failed to perform local mapping on the lookup node for plan: [" + physicalPlan + "]", e);
+ return null;
+ }
+ }
+ return null;
}
@Override
@@ -131,6 +162,8 @@ protected AbstractLookupService.LookupResponse readLookupResponse(StreamInput in
public static class Request extends AbstractLookupService.Request {
private final List matchFields;
+ private final PhysicalPlan rightPreJoinPlan;
+ private final Expression joinOnConditions;
Request(
String sessionId,
@@ -139,10 +172,14 @@ public static class Request extends AbstractLookupService.Request {
List matchFields,
Page inputPage,
List extractFields,
- Source source
+ Source source,
+ PhysicalPlan rightPreJoinPlan,
+ Expression joinOnConditions
) {
super(sessionId, index, indexPattern, matchFields.get(0).type(), inputPage, extractFields, source);
this.matchFields = matchFields;
+ this.rightPreJoinPlan = rightPreJoinPlan;
+ this.joinOnConditions = joinOnConditions;
}
}
@@ -153,6 +190,8 @@ protected static class TransportRequest extends AbstractLookupService.TransportR
);
private final List matchFields;
+ private final PhysicalPlan rightPreJoinPlan;
+ private final Expression joinOnConditions;
// Right now we assume that the page contains the same number of blocks as matchFields and that the blocks are in the same order
// The channel information inside the MatchConfig, should say the same thing
@@ -164,10 +203,14 @@ protected static class TransportRequest extends AbstractLookupService.TransportR
Page toRelease,
List extractFields,
List matchFields,
- Source source
+ Source source,
+ PhysicalPlan rightPreJoinPlan,
+ Expression joinOnConditions
) {
super(sessionId, shardId, indexPattern, inputPage, toRelease, extractFields, source);
this.matchFields = matchFields;
+ this.rightPreJoinPlan = rightPreJoinPlan;
+ this.joinOnConditions = joinOnConditions;
}
static TransportRequest readFrom(StreamInput in, BlockFactory blockFactory) throws IOException {
@@ -213,6 +256,14 @@ static TransportRequest readFrom(StreamInput in, BlockFactory blockFactory) thro
String sourceText = in.readString();
source = new Source(source.source(), sourceText);
}
+ PhysicalPlan rightPreJoinPlan = null;
+ if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_PRE_JOIN_FILTER)) {
+ rightPreJoinPlan = planIn.readOptionalNamedWriteable(PhysicalPlan.class);
+ }
+ Expression joinOnConditions = null;
+ if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ joinOnConditions = in.readOptionalNamedWriteable(Expression.class);
+ }
TransportRequest result = new TransportRequest(
sessionId,
shardId,
@@ -221,12 +272,22 @@ static TransportRequest readFrom(StreamInput in, BlockFactory blockFactory) thro
inputPage,
extractFields,
matchFields,
- source
+ source,
+ rightPreJoinPlan,
+ joinOnConditions
);
result.setParentTask(parentTaskId);
return result;
}
+ public Expression getJoinOnConditions() {
+ return joinOnConditions;
+ }
+
+ public List getMatchFields() {
+ return matchFields;
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
@@ -264,11 +325,24 @@ public void writeTo(StreamOutput out) throws IOException {
if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_SOURCE_TEXT)) {
out.writeString(source.text());
}
+ if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_PRE_JOIN_FILTER)) {
+ planOut.writeOptionalNamedWriteable(rightPreJoinPlan);
+ }
+ if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ out.writeOptionalNamedWriteable(joinOnConditions);
+ } else {
+ if (joinOnConditions != null) {
+ throw new EsqlIllegalArgumentException("LOOKUP JOIN with ON conditions is not supported on remote node");
+ }
+ }
}
@Override
protected String extraDescription() {
- return " ,match_fields=" + matchFields.stream().map(x -> x.fieldName().string()).collect(Collectors.joining(", "));
+ return " ,match_fields="
+ + matchFields.stream().map(x -> x.fieldName().string()).collect(Collectors.joining(", "))
+ + ", right_pre_join_plan="
+ + (rightPreJoinPlan == null ? "null" : rightPreJoinPlan.toString());
}
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/MatchConfig.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/MatchConfig.java
index 616c6710eff48..4690c22d52d42 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/MatchConfig.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/MatchConfig.java
@@ -28,10 +28,10 @@ public MatchConfig(FieldAttribute.FieldName fieldName, int channel, DataType typ
this.type = type;
}
- public MatchConfig(FieldAttribute match, Layout.ChannelAndType input) {
+ public MatchConfig(FieldAttribute.FieldName fieldName, Layout.ChannelAndType input) {
// TODO: Using exactAttribute was supposed to handle TEXT fields with KEYWORD subfields - but we don't allow these in lookup
// indices, so the call to exactAttribute looks redundant now.
- this(match.exactAttribute().fieldName(), input.channel(), input.type());
+ this(fieldName, input.channel(), input.type());
}
public MatchConfig(StreamInput in) throws IOException {
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java
index 692bab7d653b1..7a38ae14f87e3 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFilters.java
@@ -13,9 +13,12 @@
import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.expression.Expressions;
+import org.elasticsearch.xpack.esql.core.expression.FoldContext;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
import org.elasticsearch.xpack.esql.expression.predicate.Predicates;
+import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext;
import org.elasticsearch.xpack.esql.plan.logical.Enrich;
import org.elasticsearch.xpack.esql.plan.logical.Eval;
import org.elasticsearch.xpack.esql.plan.logical.Filter;
@@ -45,9 +48,14 @@
*
* Also combines adjacent filters using a logical {@code AND}.
*/
-public final class PushDownAndCombineFilters extends OptimizerRules.OptimizerRule {
+public final class PushDownAndCombineFilters extends OptimizerRules.ParameterizedOptimizerRule {
+
+ public PushDownAndCombineFilters() {
+ super(OptimizerRules.TransformDirection.DOWN);
+ }
+
@Override
- protected LogicalPlan rule(Filter filter) {
+ protected LogicalPlan rule(Filter filter, LogicalOptimizerContext ctx) {
LogicalPlan plan = filter;
LogicalPlan child = filter.child();
Expression condition = filter.condition();
@@ -100,7 +108,7 @@ protected LogicalPlan rule(Filter filter) {
// TODO: could we do better here about pushing down filters for inlinestats?
// See also https://github.com/elastic/elasticsearch/issues/127497
// Push down past INLINESTATS if the condition is on the groupings
- return pushDownPastJoin(filter, join);
+ return pushDownPastJoin(filter, join, ctx.foldCtx());
}
// cannot push past a Limit, this could change the tailing result set returned
return plan;
@@ -127,7 +135,7 @@ private static ScopedFilter scopeFilter(List filters, LogicalPlan le
return new ScopedFilter(rest, leftFilters, rightFilters);
}
- private static LogicalPlan pushDownPastJoin(Filter filter, Join join) {
+ private static LogicalPlan pushDownPastJoin(Filter filter, Join join, FoldContext foldCtx) {
LogicalPlan plan = filter;
// pushdown only through LEFT joins
// TODO: generalize this for other join types
@@ -140,14 +148,66 @@ private static LogicalPlan pushDownPastJoin(Filter filter, Join join) {
// 2. filter scoped to the right
// 3. filter that requires both sides to be evaluated
ScopedFilter scoped = scopeFilter(Predicates.splitAnd(filter.condition()), left, right);
- // push the left scoped filter down to the left child, keep the rest intact
+ boolean optimizationApplied = false;
+ // push the left scoped filter down to the left child
if (scoped.leftFilters.size() > 0) {
// push the filter down to the left child
left = new Filter(left.source(), left, Predicates.combineAnd(scoped.leftFilters));
// update the join with the new left child
join = (Join) join.replaceLeft(left);
+ // we completely applied the left filters, so we can remove them from the scoped filters
+ scoped = new ScopedFilter(scoped.commonFilters(), List.of(), scoped.rightFilters);
+ optimizationApplied = true;
+ }
+ // push the right scoped filter down to the right child
+ // We check if each AND component of the filter is already part of the right side filter before we add it
+ // In the future, this optimization can apply to other types of joins as well such as InlineJoin
+ // but for now we limit it to LEFT joins only, till filters are supported for other join types
+ if (scoped.rightFilters().isEmpty() == false) {
+ List rightPushableFilters = buildRightPushableFilters(scoped.rightFilters(), foldCtx);
+ if (rightPushableFilters.isEmpty() == false) {
+ if (join.right() instanceof Filter existingRightFilter) {
+ // merge the unique AND filter components from rightPushableFilters and existingRightFilter.condition()
+
+ List existingFilters = new ArrayList<>(Predicates.splitAnd(existingRightFilter.condition()));
+ int sizeBefore = existingFilters.size();
+ rightPushableFilters.stream().filter(e -> existingFilters.contains(e) == false).forEach(existingFilters::add);
+ if (sizeBefore != existingFilters.size()) {
+ right = existingRightFilter.with(Predicates.combineAnd(existingFilters));
+ join = (Join) join.replaceRight(right);
+ optimizationApplied = true;
+ } // else nothing needs to be updated
+ } else {
+ // create a new filter on top of the right child
+ right = new Filter(right.source(), right, Predicates.combineAnd(rightPushableFilters));
+ // update the join with the new right child
+ join = (Join) join.replaceRight(right);
+ optimizationApplied = true;
+ }
+ }
+ /*
+ We still want to reapply the filters that we just applied to the right child,
+ so we do NOT update scoped, and we do NOT mark optimizationApplied as true.
+ This is because by pushing them on the right side, we filter what rows we get from the right side
+ But we do not limit the output rows of the join as the rows are kept as not matched on the left side
+ So we end up applying the right filters twice, once on the right side and once on top of the join
+ This will result in major performance optimization when the lookup join is expanding
+ and applying the right filters reduces the expansion significantly.
+ For example, consider an expanding lookup join of 100,000 rows table with 10,000 lookup table
+ with filter of selectivity 0.1% on the right side(keeps 10 out of 10,000 rows of the lookup table).
+ In the non-optimized version the filter is not pushed to the right, and we get an explosion of records.
+ We have 100,000x10,000 = 1,000,000,000 rows after the join without the optimization.
+ Then we filter then out to only 1,000,000 rows.
+ With the optimization we apply the filter early so after the expanding join we only have 1,000,000 rows.
+ This reduced max number of rows used by a factor of 1,000
- // keep the remaining filters in place, otherwise return the new join;
+ In the future, once we have inner join support, it is usually possible to convert the lookup join into an inner join
+ This would allow us to not reapply the filters pushed to the right side again above the join,
+ as the inner join would only return rows that match on both sides.
+ */
+ }
+ if (optimizationApplied) {
+ // if we pushed down some filters, we need to update the filters to reapply above the join
Expression remainingFilter = Predicates.combineAnd(CollectionUtils.combine(scoped.commonFilters, scoped.rightFilters));
plan = remainingFilter != null ? filter.with(join, remainingFilter) : join;
}
@@ -156,6 +216,38 @@ private static LogicalPlan pushDownPastJoin(Filter filter, Join join) {
return plan;
}
+ /**
+ * Builds the right pushable filters for the given expressions.
+ */
+ private static List buildRightPushableFilters(List expressions, FoldContext foldCtx) {
+ return expressions.stream().filter(x -> isRightPushableFilter(x, foldCtx)).toList();
+ }
+
+ /**
+ * Determines if the given expression can be pushed down to the right side of a join.
+ * A filter is right pushable if the filter's predicate evaluates to false or null when all fields are set to null
+ * This rule helps us guard against the case where we don't know if a field is null because:
+ * 1. the field is null in the source data or
+ * 2. the field is null because there was no match in the join
+ * If the null could be an issue we just say the filter is not pushable and we avoid this issue
+ * In this context pushable means that we can push the filter down to the right side of a LEFT join
+ * We do not check if the filter is pushable to Lucene or not here
+ */
+ private static boolean isRightPushableFilter(Expression filter, FoldContext foldCtx) {
+ // traverse the filter tree
+ // replace any reference to an attribute with a null literal
+ Expression nullifiedFilter = filter.transformUp(Attribute.class, r -> new Literal(r.source(), null, r.dataType()));
+ // try to fold the filter
+ // check if the folded filter evaluates to false or null, if yes return true
+ // pushable WHERE field > 1 (evaluates to null), WHERE field is NOT NULL (evaluates to false)
+ // not pushable WHERE field is NULL (evaluates to true), WHERE coalesce(field, 10) == 10 (evaluates to true)
+ if (nullifiedFilter.foldable()) {
+ Object folded = nullifiedFilter.fold(foldCtx);
+ return folded == null || Boolean.FALSE.equals(folded);
+ }
+ return false;
+ }
+
private static Function NO_OP = expression -> expression;
private static LogicalPlan maybePushDownPastUnary(
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java
index 004f421c49ad0..14da8082ffd05 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProject.java
@@ -49,7 +49,7 @@ protected LogicalPlan rule(Join join) {
return join;
}
- if (join.left() instanceof Project project && join.config().type() == JoinTypes.LEFT) {
+ if (join.left() instanceof Project project && join.config().type() == JoinTypes.LEFT && join.config().joinOnConditions() == null) {
AttributeMap.Builder aliasBuilder = AttributeMap.builder();
project.forEachExpression(Alias.class, a -> aliasBuilder.put(a.toAttribute(), a.child()));
var aliasesFromProject = aliasBuilder.build();
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
index 4923a775c88b3..74189a783c56d 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.interp
@@ -345,6 +345,7 @@ booleanExpression
regexBooleanExpression
matchBooleanExpression
valueExpression
+comparisonExpression
operatorExpression
primaryExpression
functionExpression
@@ -362,8 +363,7 @@ comparisonOperator
joinCommand
joinTarget
joinCondition
-joinPredicate
atn:
-[4, 1, 135, 867, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 2, 46, 7, 46, 2, 47, 7, 47, 2, 48, 7, 48, 2, 49, 7, 49, 2, 50, 7, 50, 2, 51, 7, 51, 2, 52, 7, 52, 2, 53, 7, 53, 2, 54, 7, 54, 2, 55, 7, 55, 2, 56, 7, 56, 2, 57, 7, 57, 2, 58, 7, 58, 2, 59, 7, 59, 2, 60, 7, 60, 2, 61, 7, 61, 2, 62, 7, 62, 2, 63, 7, 63, 2, 64, 7, 64, 2, 65, 7, 65, 2, 66, 7, 66, 2, 67, 7, 67, 2, 68, 7, 68, 2, 69, 7, 69, 2, 70, 7, 70, 2, 71, 7, 71, 2, 72, 7, 72, 2, 73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 2, 76, 7, 76, 2, 77, 7, 77, 2, 78, 7, 78, 2, 79, 7, 79, 2, 80, 7, 80, 2, 81, 7, 81, 2, 82, 7, 82, 2, 83, 7, 83, 2, 84, 7, 84, 2, 85, 7, 85, 2, 86, 7, 86, 2, 87, 7, 87, 1, 0, 1, 0, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 5, 1, 186, 8, 1, 10, 1, 12, 1, 189, 9, 1, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 3, 2, 198, 8, 2, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 3, 3, 226, 8, 3, 1, 4, 1, 4, 1, 4, 1, 5, 1, 5, 1, 6, 1, 6, 1, 6, 1, 7, 1, 7, 1, 7, 5, 7, 239, 8, 7, 10, 7, 12, 7, 242, 9, 7, 1, 8, 1, 8, 1, 8, 3, 8, 247, 8, 8, 1, 8, 1, 8, 1, 9, 1, 9, 1, 9, 5, 9, 254, 8, 9, 10, 9, 12, 9, 257, 9, 9, 1, 10, 1, 10, 1, 10, 3, 10, 262, 8, 10, 1, 11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 1, 13, 1, 13, 1, 13, 5, 13, 273, 8, 13, 10, 13, 12, 13, 276, 9, 13, 1, 13, 3, 13, 279, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 3, 14, 290, 8, 14, 1, 15, 1, 15, 1, 16, 1, 16, 1, 17, 1, 17, 1, 18, 1, 18, 1, 19, 1, 19, 1, 19, 1, 19, 5, 19, 304, 8, 19, 10, 19, 12, 19, 307, 9, 19, 1, 20, 1, 20, 1, 20, 1, 21, 1, 21, 3, 21, 314, 8, 21, 1, 21, 1, 21, 3, 21, 318, 8, 21, 1, 22, 1, 22, 1, 22, 5, 22, 323, 8, 22, 10, 22, 12, 22, 326, 9, 22, 1, 23, 1, 23, 1, 23, 3, 23, 331, 8, 23, 1, 24, 1, 24, 1, 24, 3, 24, 336, 8, 24, 1, 24, 1, 24, 1, 24, 1, 24, 1, 24, 1, 24, 1, 24, 3, 24, 345, 8, 24, 1, 25, 1, 25, 1, 25, 5, 25, 350, 8, 25, 10, 25, 12, 25, 353, 9, 25, 1, 26, 1, 26, 1, 26, 3, 26, 358, 8, 26, 1, 26, 1, 26, 1, 26, 1, 26, 1, 26, 1, 26, 1, 26, 3, 26, 367, 8, 26, 1, 27, 1, 27, 1, 27, 5, 27, 372, 8, 27, 10, 27, 12, 27, 375, 9, 27, 1, 28, 1, 28, 1, 28, 5, 28, 380, 8, 28, 10, 28, 12, 28, 383, 9, 28, 1, 29, 1, 29, 1, 30, 1, 30, 1, 30, 3, 30, 390, 8, 30, 1, 31, 1, 31, 3, 31, 394, 8, 31, 1, 32, 1, 32, 3, 32, 398, 8, 32, 1, 33, 1, 33, 1, 33, 3, 33, 403, 8, 33, 1, 34, 1, 34, 1, 34, 1, 35, 1, 35, 1, 35, 1, 35, 5, 35, 412, 8, 35, 10, 35, 12, 35, 415, 9, 35, 1, 36, 1, 36, 3, 36, 419, 8, 36, 1, 36, 1, 36, 3, 36, 423, 8, 36, 1, 37, 1, 37, 1, 37, 1, 38, 1, 38, 1, 38, 1, 39, 1, 39, 1, 39, 1, 39, 5, 39, 435, 8, 39, 10, 39, 12, 39, 438, 9, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 3, 40, 448, 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 3, 41, 454, 8, 41, 1, 42, 1, 42, 1, 42, 5, 42, 459, 8, 42, 10, 42, 12, 42, 462, 9, 42, 1, 43, 1, 43, 1, 43, 1, 43, 1, 44, 1, 44, 3, 44, 470, 8, 44, 1, 45, 1, 45, 1, 45, 1, 45, 1, 46, 1, 46, 1, 46, 1, 47, 1, 47, 1, 47, 1, 48, 1, 48, 1, 48, 1, 48, 1, 49, 1, 49, 1, 49, 1, 50, 1, 50, 1, 50, 1, 50, 3, 50, 493, 8, 50, 1, 50, 1, 50, 1, 50, 1, 50, 5, 50, 499, 8, 50, 10, 50, 12, 50, 502, 9, 50, 3, 50, 504, 8, 50, 1, 51, 1, 51, 1, 52, 1, 52, 1, 52, 3, 52, 511, 8, 52, 1, 52, 1, 52, 1, 53, 1, 53, 1, 53, 1, 54, 1, 54, 1, 54, 1, 54, 3, 54, 522, 8, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 3, 54, 529, 8, 54, 1, 55, 1, 55, 1, 55, 1, 56, 4, 56, 535, 8, 56, 11, 56, 12, 56, 536, 1, 57, 1, 57, 1, 57, 1, 57, 1, 58, 1, 58, 1, 58, 1, 58, 1, 58, 1, 58, 5, 58, 549, 8, 58, 10, 58, 12, 58, 552, 9, 58, 1, 59, 1, 59, 1, 60, 1, 60, 1, 60, 1, 60, 3, 60, 560, 8, 60, 1, 60, 1, 60, 1, 60, 1, 60, 1, 60, 1, 61, 1, 61, 1, 61, 1, 61, 3, 61, 571, 8, 61, 1, 61, 1, 61, 1, 61, 1, 62, 1, 62, 1, 62, 1, 62, 1, 62, 1, 63, 1, 63, 1, 63, 1, 63, 3, 63, 585, 8, 63, 1, 64, 1, 64, 1, 64, 1, 65, 1, 65, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 3, 66, 599, 8, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 5, 66, 606, 8, 66, 10, 66, 12, 66, 609, 9, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 3, 66, 616, 8, 66, 1, 66, 1, 66, 1, 66, 3, 66, 621, 8, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 5, 66, 629, 8, 66, 10, 66, 12, 66, 632, 9, 66, 1, 67, 1, 67, 3, 67, 636, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 643, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 650, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 5, 67, 657, 8, 67, 10, 67, 12, 67, 660, 9, 67, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 666, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 5, 67, 673, 8, 67, 10, 67, 12, 67, 676, 9, 67, 1, 67, 1, 67, 3, 67, 680, 8, 67, 1, 68, 1, 68, 1, 68, 3, 68, 685, 8, 68, 1, 68, 1, 68, 1, 68, 1, 69, 1, 69, 1, 69, 1, 69, 1, 69, 3, 69, 695, 8, 69, 1, 70, 1, 70, 1, 70, 1, 70, 3, 70, 701, 8, 70, 1, 70, 1, 70, 1, 70, 1, 70, 1, 70, 1, 70, 5, 70, 709, 8, 70, 10, 70, 12, 70, 712, 9, 70, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 3, 71, 722, 8, 71, 1, 71, 1, 71, 1, 71, 5, 71, 727, 8, 71, 10, 71, 12, 71, 730, 9, 71, 1, 72, 1, 72, 1, 72, 1, 72, 1, 72, 1, 72, 5, 72, 738, 8, 72, 10, 72, 12, 72, 741, 9, 72, 1, 72, 1, 72, 3, 72, 745, 8, 72, 3, 72, 747, 8, 72, 1, 72, 1, 72, 1, 73, 1, 73, 1, 73, 3, 73, 754, 8, 73, 1, 74, 1, 74, 1, 74, 1, 74, 5, 74, 760, 8, 74, 10, 74, 12, 74, 763, 9, 74, 3, 74, 765, 8, 74, 1, 74, 1, 74, 1, 75, 1, 75, 1, 75, 1, 75, 1, 76, 1, 76, 3, 76, 775, 8, 76, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 5, 77, 790, 8, 77, 10, 77, 12, 77, 793, 9, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 5, 77, 801, 8, 77, 10, 77, 12, 77, 804, 9, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 1, 77, 5, 77, 812, 8, 77, 10, 77, 12, 77, 815, 9, 77, 1, 77, 1, 77, 3, 77, 819, 8, 77, 1, 78, 1, 78, 1, 79, 1, 79, 3, 79, 825, 8, 79, 1, 80, 3, 80, 828, 8, 80, 1, 80, 1, 80, 1, 81, 3, 81, 833, 8, 81, 1, 81, 1, 81, 1, 82, 1, 82, 1, 83, 1, 83, 1, 84, 1, 84, 1, 84, 1, 84, 1, 84, 1, 85, 1, 85, 1, 85, 3, 85, 849, 8, 85, 1, 85, 1, 85, 1, 85, 3, 85, 854, 8, 85, 1, 86, 1, 86, 1, 86, 1, 86, 5, 86, 860, 8, 86, 10, 86, 12, 86, 863, 9, 86, 1, 87, 1, 87, 1, 87, 0, 5, 2, 116, 132, 140, 142, 88, 0, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20, 22, 24, 26, 28, 30, 32, 34, 36, 38, 40, 42, 44, 46, 48, 50, 52, 54, 56, 58, 60, 62, 64, 66, 68, 70, 72, 74, 76, 78, 80, 82, 84, 86, 88, 90, 92, 94, 96, 98, 100, 102, 104, 106, 108, 110, 112, 114, 116, 118, 120, 122, 124, 126, 128, 130, 132, 134, 136, 138, 140, 142, 144, 146, 148, 150, 152, 154, 156, 158, 160, 162, 164, 166, 168, 170, 172, 174, 0, 10, 2, 0, 49, 49, 103, 103, 1, 0, 97, 98, 2, 0, 53, 53, 59, 59, 2, 0, 62, 62, 65, 65, 2, 0, 38, 38, 49, 49, 1, 0, 83, 84, 1, 0, 85, 87, 2, 0, 61, 61, 74, 74, 2, 0, 76, 76, 78, 82, 2, 0, 23, 23, 25, 26, 904, 0, 176, 1, 0, 0, 0, 2, 179, 1, 0, 0, 0, 4, 197, 1, 0, 0, 0, 6, 225, 1, 0, 0, 0, 8, 227, 1, 0, 0, 0, 10, 230, 1, 0, 0, 0, 12, 232, 1, 0, 0, 0, 14, 235, 1, 0, 0, 0, 16, 246, 1, 0, 0, 0, 18, 250, 1, 0, 0, 0, 20, 258, 1, 0, 0, 0, 22, 263, 1, 0, 0, 0, 24, 266, 1, 0, 0, 0, 26, 269, 1, 0, 0, 0, 28, 289, 1, 0, 0, 0, 30, 291, 1, 0, 0, 0, 32, 293, 1, 0, 0, 0, 34, 295, 1, 0, 0, 0, 36, 297, 1, 0, 0, 0, 38, 299, 1, 0, 0, 0, 40, 308, 1, 0, 0, 0, 42, 311, 1, 0, 0, 0, 44, 319, 1, 0, 0, 0, 46, 327, 1, 0, 0, 0, 48, 344, 1, 0, 0, 0, 50, 346, 1, 0, 0, 0, 52, 366, 1, 0, 0, 0, 54, 368, 1, 0, 0, 0, 56, 376, 1, 0, 0, 0, 58, 384, 1, 0, 0, 0, 60, 389, 1, 0, 0, 0, 62, 393, 1, 0, 0, 0, 64, 397, 1, 0, 0, 0, 66, 402, 1, 0, 0, 0, 68, 404, 1, 0, 0, 0, 70, 407, 1, 0, 0, 0, 72, 416, 1, 0, 0, 0, 74, 424, 1, 0, 0, 0, 76, 427, 1, 0, 0, 0, 78, 430, 1, 0, 0, 0, 80, 447, 1, 0, 0, 0, 82, 449, 1, 0, 0, 0, 84, 455, 1, 0, 0, 0, 86, 463, 1, 0, 0, 0, 88, 469, 1, 0, 0, 0, 90, 471, 1, 0, 0, 0, 92, 475, 1, 0, 0, 0, 94, 478, 1, 0, 0, 0, 96, 481, 1, 0, 0, 0, 98, 485, 1, 0, 0, 0, 100, 488, 1, 0, 0, 0, 102, 505, 1, 0, 0, 0, 104, 510, 1, 0, 0, 0, 106, 514, 1, 0, 0, 0, 108, 517, 1, 0, 0, 0, 110, 530, 1, 0, 0, 0, 112, 534, 1, 0, 0, 0, 114, 538, 1, 0, 0, 0, 116, 542, 1, 0, 0, 0, 118, 553, 1, 0, 0, 0, 120, 555, 1, 0, 0, 0, 122, 566, 1, 0, 0, 0, 124, 575, 1, 0, 0, 0, 126, 580, 1, 0, 0, 0, 128, 586, 1, 0, 0, 0, 130, 589, 1, 0, 0, 0, 132, 620, 1, 0, 0, 0, 134, 679, 1, 0, 0, 0, 136, 681, 1, 0, 0, 0, 138, 694, 1, 0, 0, 0, 140, 700, 1, 0, 0, 0, 142, 721, 1, 0, 0, 0, 144, 731, 1, 0, 0, 0, 146, 753, 1, 0, 0, 0, 148, 755, 1, 0, 0, 0, 150, 768, 1, 0, 0, 0, 152, 774, 1, 0, 0, 0, 154, 818, 1, 0, 0, 0, 156, 820, 1, 0, 0, 0, 158, 824, 1, 0, 0, 0, 160, 827, 1, 0, 0, 0, 162, 832, 1, 0, 0, 0, 164, 836, 1, 0, 0, 0, 166, 838, 1, 0, 0, 0, 168, 840, 1, 0, 0, 0, 170, 853, 1, 0, 0, 0, 172, 855, 1, 0, 0, 0, 174, 864, 1, 0, 0, 0, 176, 177, 3, 2, 1, 0, 177, 178, 5, 0, 0, 1, 178, 1, 1, 0, 0, 0, 179, 180, 6, 1, -1, 0, 180, 181, 3, 4, 2, 0, 181, 187, 1, 0, 0, 0, 182, 183, 10, 1, 0, 0, 183, 184, 5, 48, 0, 0, 184, 186, 3, 6, 3, 0, 185, 182, 1, 0, 0, 0, 186, 189, 1, 0, 0, 0, 187, 185, 1, 0, 0, 0, 187, 188, 1, 0, 0, 0, 188, 3, 1, 0, 0, 0, 189, 187, 1, 0, 0, 0, 190, 198, 3, 22, 11, 0, 191, 198, 3, 12, 6, 0, 192, 198, 3, 98, 49, 0, 193, 194, 4, 2, 1, 0, 194, 198, 3, 24, 12, 0, 195, 196, 4, 2, 2, 0, 196, 198, 3, 94, 47, 0, 197, 190, 1, 0, 0, 0, 197, 191, 1, 0, 0, 0, 197, 192, 1, 0, 0, 0, 197, 193, 1, 0, 0, 0, 197, 195, 1, 0, 0, 0, 198, 5, 1, 0, 0, 0, 199, 226, 3, 40, 20, 0, 200, 226, 3, 8, 4, 0, 201, 226, 3, 74, 37, 0, 202, 226, 3, 68, 34, 0, 203, 226, 3, 42, 21, 0, 204, 226, 3, 70, 35, 0, 205, 226, 3, 76, 38, 0, 206, 226, 3, 78, 39, 0, 207, 226, 3, 82, 41, 0, 208, 226, 3, 90, 45, 0, 209, 226, 3, 100, 50, 0, 210, 226, 3, 92, 46, 0, 211, 226, 3, 168, 84, 0, 212, 226, 3, 108, 54, 0, 213, 226, 3, 122, 61, 0, 214, 226, 3, 106, 53, 0, 215, 226, 3, 110, 55, 0, 216, 226, 3, 120, 60, 0, 217, 218, 4, 3, 3, 0, 218, 226, 3, 126, 63, 0, 219, 220, 4, 3, 4, 0, 220, 226, 3, 124, 62, 0, 221, 222, 4, 3, 5, 0, 222, 226, 3, 128, 64, 0, 223, 224, 4, 3, 6, 0, 224, 226, 3, 130, 65, 0, 225, 199, 1, 0, 0, 0, 225, 200, 1, 0, 0, 0, 225, 201, 1, 0, 0, 0, 225, 202, 1, 0, 0, 0, 225, 203, 1, 0, 0, 0, 225, 204, 1, 0, 0, 0, 225, 205, 1, 0, 0, 0, 225, 206, 1, 0, 0, 0, 225, 207, 1, 0, 0, 0, 225, 208, 1, 0, 0, 0, 225, 209, 1, 0, 0, 0, 225, 210, 1, 0, 0, 0, 225, 211, 1, 0, 0, 0, 225, 212, 1, 0, 0, 0, 225, 213, 1, 0, 0, 0, 225, 214, 1, 0, 0, 0, 225, 215, 1, 0, 0, 0, 225, 216, 1, 0, 0, 0, 225, 217, 1, 0, 0, 0, 225, 219, 1, 0, 0, 0, 225, 221, 1, 0, 0, 0, 225, 223, 1, 0, 0, 0, 226, 7, 1, 0, 0, 0, 227, 228, 5, 17, 0, 0, 228, 229, 3, 132, 66, 0, 229, 9, 1, 0, 0, 0, 230, 231, 3, 58, 29, 0, 231, 11, 1, 0, 0, 0, 232, 233, 5, 13, 0, 0, 233, 234, 3, 14, 7, 0, 234, 13, 1, 0, 0, 0, 235, 240, 3, 16, 8, 0, 236, 237, 5, 58, 0, 0, 237, 239, 3, 16, 8, 0, 238, 236, 1, 0, 0, 0, 239, 242, 1, 0, 0, 0, 240, 238, 1, 0, 0, 0, 240, 241, 1, 0, 0, 0, 241, 15, 1, 0, 0, 0, 242, 240, 1, 0, 0, 0, 243, 244, 3, 48, 24, 0, 244, 245, 5, 54, 0, 0, 245, 247, 1, 0, 0, 0, 246, 243, 1, 0, 0, 0, 246, 247, 1, 0, 0, 0, 247, 248, 1, 0, 0, 0, 248, 249, 3, 132, 66, 0, 249, 17, 1, 0, 0, 0, 250, 255, 3, 20, 10, 0, 251, 252, 5, 58, 0, 0, 252, 254, 3, 20, 10, 0, 253, 251, 1, 0, 0, 0, 254, 257, 1, 0, 0, 0, 255, 253, 1, 0, 0, 0, 255, 256, 1, 0, 0, 0, 256, 19, 1, 0, 0, 0, 257, 255, 1, 0, 0, 0, 258, 261, 3, 48, 24, 0, 259, 260, 5, 54, 0, 0, 260, 262, 3, 132, 66, 0, 261, 259, 1, 0, 0, 0, 261, 262, 1, 0, 0, 0, 262, 21, 1, 0, 0, 0, 263, 264, 5, 19, 0, 0, 264, 265, 3, 26, 13, 0, 265, 23, 1, 0, 0, 0, 266, 267, 5, 20, 0, 0, 267, 268, 3, 26, 13, 0, 268, 25, 1, 0, 0, 0, 269, 274, 3, 28, 14, 0, 270, 271, 5, 58, 0, 0, 271, 273, 3, 28, 14, 0, 272, 270, 1, 0, 0, 0, 273, 276, 1, 0, 0, 0, 274, 272, 1, 0, 0, 0, 274, 275, 1, 0, 0, 0, 275, 278, 1, 0, 0, 0, 276, 274, 1, 0, 0, 0, 277, 279, 3, 38, 19, 0, 278, 277, 1, 0, 0, 0, 278, 279, 1, 0, 0, 0, 279, 27, 1, 0, 0, 0, 280, 281, 3, 30, 15, 0, 281, 282, 5, 57, 0, 0, 282, 283, 3, 34, 17, 0, 283, 290, 1, 0, 0, 0, 284, 285, 3, 34, 17, 0, 285, 286, 5, 56, 0, 0, 286, 287, 3, 32, 16, 0, 287, 290, 1, 0, 0, 0, 288, 290, 3, 36, 18, 0, 289, 280, 1, 0, 0, 0, 289, 284, 1, 0, 0, 0, 289, 288, 1, 0, 0, 0, 290, 29, 1, 0, 0, 0, 291, 292, 5, 103, 0, 0, 292, 31, 1, 0, 0, 0, 293, 294, 5, 103, 0, 0, 294, 33, 1, 0, 0, 0, 295, 296, 5, 103, 0, 0, 296, 35, 1, 0, 0, 0, 297, 298, 7, 0, 0, 0, 298, 37, 1, 0, 0, 0, 299, 300, 5, 102, 0, 0, 300, 305, 5, 103, 0, 0, 301, 302, 5, 58, 0, 0, 302, 304, 5, 103, 0, 0, 303, 301, 1, 0, 0, 0, 304, 307, 1, 0, 0, 0, 305, 303, 1, 0, 0, 0, 305, 306, 1, 0, 0, 0, 306, 39, 1, 0, 0, 0, 307, 305, 1, 0, 0, 0, 308, 309, 5, 9, 0, 0, 309, 310, 3, 14, 7, 0, 310, 41, 1, 0, 0, 0, 311, 313, 5, 16, 0, 0, 312, 314, 3, 44, 22, 0, 313, 312, 1, 0, 0, 0, 313, 314, 1, 0, 0, 0, 314, 317, 1, 0, 0, 0, 315, 316, 5, 55, 0, 0, 316, 318, 3, 14, 7, 0, 317, 315, 1, 0, 0, 0, 317, 318, 1, 0, 0, 0, 318, 43, 1, 0, 0, 0, 319, 324, 3, 46, 23, 0, 320, 321, 5, 58, 0, 0, 321, 323, 3, 46, 23, 0, 322, 320, 1, 0, 0, 0, 323, 326, 1, 0, 0, 0, 324, 322, 1, 0, 0, 0, 324, 325, 1, 0, 0, 0, 325, 45, 1, 0, 0, 0, 326, 324, 1, 0, 0, 0, 327, 330, 3, 16, 8, 0, 328, 329, 5, 17, 0, 0, 329, 331, 3, 132, 66, 0, 330, 328, 1, 0, 0, 0, 330, 331, 1, 0, 0, 0, 331, 47, 1, 0, 0, 0, 332, 333, 4, 24, 7, 0, 333, 335, 5, 93, 0, 0, 334, 336, 5, 97, 0, 0, 335, 334, 1, 0, 0, 0, 335, 336, 1, 0, 0, 0, 336, 337, 1, 0, 0, 0, 337, 338, 5, 94, 0, 0, 338, 339, 5, 60, 0, 0, 339, 340, 5, 93, 0, 0, 340, 341, 3, 50, 25, 0, 341, 342, 5, 94, 0, 0, 342, 345, 1, 0, 0, 0, 343, 345, 3, 50, 25, 0, 344, 332, 1, 0, 0, 0, 344, 343, 1, 0, 0, 0, 345, 49, 1, 0, 0, 0, 346, 351, 3, 66, 33, 0, 347, 348, 5, 60, 0, 0, 348, 350, 3, 66, 33, 0, 349, 347, 1, 0, 0, 0, 350, 353, 1, 0, 0, 0, 351, 349, 1, 0, 0, 0, 351, 352, 1, 0, 0, 0, 352, 51, 1, 0, 0, 0, 353, 351, 1, 0, 0, 0, 354, 355, 4, 26, 8, 0, 355, 357, 5, 93, 0, 0, 356, 358, 5, 124, 0, 0, 357, 356, 1, 0, 0, 0, 357, 358, 1, 0, 0, 0, 358, 359, 1, 0, 0, 0, 359, 360, 5, 94, 0, 0, 360, 361, 5, 60, 0, 0, 361, 362, 5, 93, 0, 0, 362, 363, 3, 54, 27, 0, 363, 364, 5, 94, 0, 0, 364, 367, 1, 0, 0, 0, 365, 367, 3, 54, 27, 0, 366, 354, 1, 0, 0, 0, 366, 365, 1, 0, 0, 0, 367, 53, 1, 0, 0, 0, 368, 373, 3, 60, 30, 0, 369, 370, 5, 60, 0, 0, 370, 372, 3, 60, 30, 0, 371, 369, 1, 0, 0, 0, 372, 375, 1, 0, 0, 0, 373, 371, 1, 0, 0, 0, 373, 374, 1, 0, 0, 0, 374, 55, 1, 0, 0, 0, 375, 373, 1, 0, 0, 0, 376, 381, 3, 52, 26, 0, 377, 378, 5, 58, 0, 0, 378, 380, 3, 52, 26, 0, 379, 377, 1, 0, 0, 0, 380, 383, 1, 0, 0, 0, 381, 379, 1, 0, 0, 0, 381, 382, 1, 0, 0, 0, 382, 57, 1, 0, 0, 0, 383, 381, 1, 0, 0, 0, 384, 385, 7, 1, 0, 0, 385, 59, 1, 0, 0, 0, 386, 390, 5, 124, 0, 0, 387, 390, 3, 62, 31, 0, 388, 390, 3, 64, 32, 0, 389, 386, 1, 0, 0, 0, 389, 387, 1, 0, 0, 0, 389, 388, 1, 0, 0, 0, 390, 61, 1, 0, 0, 0, 391, 394, 5, 72, 0, 0, 392, 394, 5, 91, 0, 0, 393, 391, 1, 0, 0, 0, 393, 392, 1, 0, 0, 0, 394, 63, 1, 0, 0, 0, 395, 398, 5, 90, 0, 0, 396, 398, 5, 92, 0, 0, 397, 395, 1, 0, 0, 0, 397, 396, 1, 0, 0, 0, 398, 65, 1, 0, 0, 0, 399, 403, 3, 58, 29, 0, 400, 403, 3, 62, 31, 0, 401, 403, 3, 64, 32, 0, 402, 399, 1, 0, 0, 0, 402, 400, 1, 0, 0, 0, 402, 401, 1, 0, 0, 0, 403, 67, 1, 0, 0, 0, 404, 405, 5, 11, 0, 0, 405, 406, 3, 154, 77, 0, 406, 69, 1, 0, 0, 0, 407, 408, 5, 15, 0, 0, 408, 413, 3, 72, 36, 0, 409, 410, 5, 58, 0, 0, 410, 412, 3, 72, 36, 0, 411, 409, 1, 0, 0, 0, 412, 415, 1, 0, 0, 0, 413, 411, 1, 0, 0, 0, 413, 414, 1, 0, 0, 0, 414, 71, 1, 0, 0, 0, 415, 413, 1, 0, 0, 0, 416, 418, 3, 132, 66, 0, 417, 419, 7, 2, 0, 0, 418, 417, 1, 0, 0, 0, 418, 419, 1, 0, 0, 0, 419, 422, 1, 0, 0, 0, 420, 421, 5, 69, 0, 0, 421, 423, 7, 3, 0, 0, 422, 420, 1, 0, 0, 0, 422, 423, 1, 0, 0, 0, 423, 73, 1, 0, 0, 0, 424, 425, 5, 30, 0, 0, 425, 426, 3, 56, 28, 0, 426, 75, 1, 0, 0, 0, 427, 428, 5, 29, 0, 0, 428, 429, 3, 56, 28, 0, 429, 77, 1, 0, 0, 0, 430, 431, 5, 32, 0, 0, 431, 436, 3, 80, 40, 0, 432, 433, 5, 58, 0, 0, 433, 435, 3, 80, 40, 0, 434, 432, 1, 0, 0, 0, 435, 438, 1, 0, 0, 0, 436, 434, 1, 0, 0, 0, 436, 437, 1, 0, 0, 0, 437, 79, 1, 0, 0, 0, 438, 436, 1, 0, 0, 0, 439, 440, 3, 52, 26, 0, 440, 441, 5, 128, 0, 0, 441, 442, 3, 52, 26, 0, 442, 448, 1, 0, 0, 0, 443, 444, 3, 52, 26, 0, 444, 445, 5, 54, 0, 0, 445, 446, 3, 52, 26, 0, 446, 448, 1, 0, 0, 0, 447, 439, 1, 0, 0, 0, 447, 443, 1, 0, 0, 0, 448, 81, 1, 0, 0, 0, 449, 450, 5, 8, 0, 0, 450, 451, 3, 142, 71, 0, 451, 453, 3, 164, 82, 0, 452, 454, 3, 84, 42, 0, 453, 452, 1, 0, 0, 0, 453, 454, 1, 0, 0, 0, 454, 83, 1, 0, 0, 0, 455, 460, 3, 86, 43, 0, 456, 457, 5, 58, 0, 0, 457, 459, 3, 86, 43, 0, 458, 456, 1, 0, 0, 0, 459, 462, 1, 0, 0, 0, 460, 458, 1, 0, 0, 0, 460, 461, 1, 0, 0, 0, 461, 85, 1, 0, 0, 0, 462, 460, 1, 0, 0, 0, 463, 464, 3, 58, 29, 0, 464, 465, 5, 54, 0, 0, 465, 466, 3, 154, 77, 0, 466, 87, 1, 0, 0, 0, 467, 468, 5, 75, 0, 0, 468, 470, 3, 148, 74, 0, 469, 467, 1, 0, 0, 0, 469, 470, 1, 0, 0, 0, 470, 89, 1, 0, 0, 0, 471, 472, 5, 10, 0, 0, 472, 473, 3, 142, 71, 0, 473, 474, 3, 164, 82, 0, 474, 91, 1, 0, 0, 0, 475, 476, 5, 28, 0, 0, 476, 477, 3, 48, 24, 0, 477, 93, 1, 0, 0, 0, 478, 479, 5, 6, 0, 0, 479, 480, 3, 96, 48, 0, 480, 95, 1, 0, 0, 0, 481, 482, 5, 95, 0, 0, 482, 483, 3, 2, 1, 0, 483, 484, 5, 96, 0, 0, 484, 97, 1, 0, 0, 0, 485, 486, 5, 33, 0, 0, 486, 487, 5, 132, 0, 0, 487, 99, 1, 0, 0, 0, 488, 489, 5, 5, 0, 0, 489, 492, 3, 102, 51, 0, 490, 491, 5, 70, 0, 0, 491, 493, 3, 52, 26, 0, 492, 490, 1, 0, 0, 0, 492, 493, 1, 0, 0, 0, 493, 503, 1, 0, 0, 0, 494, 495, 5, 75, 0, 0, 495, 500, 3, 104, 52, 0, 496, 497, 5, 58, 0, 0, 497, 499, 3, 104, 52, 0, 498, 496, 1, 0, 0, 0, 499, 502, 1, 0, 0, 0, 500, 498, 1, 0, 0, 0, 500, 501, 1, 0, 0, 0, 501, 504, 1, 0, 0, 0, 502, 500, 1, 0, 0, 0, 503, 494, 1, 0, 0, 0, 503, 504, 1, 0, 0, 0, 504, 101, 1, 0, 0, 0, 505, 506, 7, 4, 0, 0, 506, 103, 1, 0, 0, 0, 507, 508, 3, 52, 26, 0, 508, 509, 5, 54, 0, 0, 509, 511, 1, 0, 0, 0, 510, 507, 1, 0, 0, 0, 510, 511, 1, 0, 0, 0, 511, 512, 1, 0, 0, 0, 512, 513, 3, 52, 26, 0, 513, 105, 1, 0, 0, 0, 514, 515, 5, 14, 0, 0, 515, 516, 3, 154, 77, 0, 516, 107, 1, 0, 0, 0, 517, 518, 5, 4, 0, 0, 518, 521, 3, 48, 24, 0, 519, 520, 5, 70, 0, 0, 520, 522, 3, 48, 24, 0, 521, 519, 1, 0, 0, 0, 521, 522, 1, 0, 0, 0, 522, 528, 1, 0, 0, 0, 523, 524, 5, 128, 0, 0, 524, 525, 3, 48, 24, 0, 525, 526, 5, 58, 0, 0, 526, 527, 3, 48, 24, 0, 527, 529, 1, 0, 0, 0, 528, 523, 1, 0, 0, 0, 528, 529, 1, 0, 0, 0, 529, 109, 1, 0, 0, 0, 530, 531, 5, 21, 0, 0, 531, 532, 3, 112, 56, 0, 532, 111, 1, 0, 0, 0, 533, 535, 3, 114, 57, 0, 534, 533, 1, 0, 0, 0, 535, 536, 1, 0, 0, 0, 536, 534, 1, 0, 0, 0, 536, 537, 1, 0, 0, 0, 537, 113, 1, 0, 0, 0, 538, 539, 5, 95, 0, 0, 539, 540, 3, 116, 58, 0, 540, 541, 5, 96, 0, 0, 541, 115, 1, 0, 0, 0, 542, 543, 6, 58, -1, 0, 543, 544, 3, 118, 59, 0, 544, 550, 1, 0, 0, 0, 545, 546, 10, 1, 0, 0, 546, 547, 5, 48, 0, 0, 547, 549, 3, 118, 59, 0, 548, 545, 1, 0, 0, 0, 549, 552, 1, 0, 0, 0, 550, 548, 1, 0, 0, 0, 550, 551, 1, 0, 0, 0, 551, 117, 1, 0, 0, 0, 552, 550, 1, 0, 0, 0, 553, 554, 3, 6, 3, 0, 554, 119, 1, 0, 0, 0, 555, 559, 5, 12, 0, 0, 556, 557, 3, 48, 24, 0, 557, 558, 5, 54, 0, 0, 558, 560, 1, 0, 0, 0, 559, 556, 1, 0, 0, 0, 559, 560, 1, 0, 0, 0, 560, 561, 1, 0, 0, 0, 561, 562, 3, 154, 77, 0, 562, 563, 5, 70, 0, 0, 563, 564, 3, 18, 9, 0, 564, 565, 3, 88, 44, 0, 565, 121, 1, 0, 0, 0, 566, 570, 5, 7, 0, 0, 567, 568, 3, 48, 24, 0, 568, 569, 5, 54, 0, 0, 569, 571, 1, 0, 0, 0, 570, 567, 1, 0, 0, 0, 570, 571, 1, 0, 0, 0, 571, 572, 1, 0, 0, 0, 572, 573, 3, 142, 71, 0, 573, 574, 3, 88, 44, 0, 574, 123, 1, 0, 0, 0, 575, 576, 5, 27, 0, 0, 576, 577, 3, 28, 14, 0, 577, 578, 5, 70, 0, 0, 578, 579, 3, 56, 28, 0, 579, 125, 1, 0, 0, 0, 580, 581, 5, 18, 0, 0, 581, 584, 3, 44, 22, 0, 582, 583, 5, 55, 0, 0, 583, 585, 3, 14, 7, 0, 584, 582, 1, 0, 0, 0, 584, 585, 1, 0, 0, 0, 585, 127, 1, 0, 0, 0, 586, 587, 5, 31, 0, 0, 587, 588, 3, 56, 28, 0, 588, 129, 1, 0, 0, 0, 589, 590, 5, 22, 0, 0, 590, 131, 1, 0, 0, 0, 591, 592, 6, 66, -1, 0, 592, 593, 5, 67, 0, 0, 593, 621, 3, 132, 66, 8, 594, 621, 3, 138, 69, 0, 595, 621, 3, 134, 67, 0, 596, 598, 3, 138, 69, 0, 597, 599, 5, 67, 0, 0, 598, 597, 1, 0, 0, 0, 598, 599, 1, 0, 0, 0, 599, 600, 1, 0, 0, 0, 600, 601, 5, 63, 0, 0, 601, 602, 5, 95, 0, 0, 602, 607, 3, 138, 69, 0, 603, 604, 5, 58, 0, 0, 604, 606, 3, 138, 69, 0, 605, 603, 1, 0, 0, 0, 606, 609, 1, 0, 0, 0, 607, 605, 1, 0, 0, 0, 607, 608, 1, 0, 0, 0, 608, 610, 1, 0, 0, 0, 609, 607, 1, 0, 0, 0, 610, 611, 5, 96, 0, 0, 611, 621, 1, 0, 0, 0, 612, 613, 3, 138, 69, 0, 613, 615, 5, 64, 0, 0, 614, 616, 5, 67, 0, 0, 615, 614, 1, 0, 0, 0, 615, 616, 1, 0, 0, 0, 616, 617, 1, 0, 0, 0, 617, 618, 5, 68, 0, 0, 618, 621, 1, 0, 0, 0, 619, 621, 3, 136, 68, 0, 620, 591, 1, 0, 0, 0, 620, 594, 1, 0, 0, 0, 620, 595, 1, 0, 0, 0, 620, 596, 1, 0, 0, 0, 620, 612, 1, 0, 0, 0, 620, 619, 1, 0, 0, 0, 621, 630, 1, 0, 0, 0, 622, 623, 10, 5, 0, 0, 623, 624, 5, 52, 0, 0, 624, 629, 3, 132, 66, 6, 625, 626, 10, 4, 0, 0, 626, 627, 5, 71, 0, 0, 627, 629, 3, 132, 66, 5, 628, 622, 1, 0, 0, 0, 628, 625, 1, 0, 0, 0, 629, 632, 1, 0, 0, 0, 630, 628, 1, 0, 0, 0, 630, 631, 1, 0, 0, 0, 631, 133, 1, 0, 0, 0, 632, 630, 1, 0, 0, 0, 633, 635, 3, 138, 69, 0, 634, 636, 5, 67, 0, 0, 635, 634, 1, 0, 0, 0, 635, 636, 1, 0, 0, 0, 636, 637, 1, 0, 0, 0, 637, 638, 5, 66, 0, 0, 638, 639, 3, 164, 82, 0, 639, 680, 1, 0, 0, 0, 640, 642, 3, 138, 69, 0, 641, 643, 5, 67, 0, 0, 642, 641, 1, 0, 0, 0, 642, 643, 1, 0, 0, 0, 643, 644, 1, 0, 0, 0, 644, 645, 5, 73, 0, 0, 645, 646, 3, 164, 82, 0, 646, 680, 1, 0, 0, 0, 647, 649, 3, 138, 69, 0, 648, 650, 5, 67, 0, 0, 649, 648, 1, 0, 0, 0, 649, 650, 1, 0, 0, 0, 650, 651, 1, 0, 0, 0, 651, 652, 5, 66, 0, 0, 652, 653, 5, 95, 0, 0, 653, 658, 3, 164, 82, 0, 654, 655, 5, 58, 0, 0, 655, 657, 3, 164, 82, 0, 656, 654, 1, 0, 0, 0, 657, 660, 1, 0, 0, 0, 658, 656, 1, 0, 0, 0, 658, 659, 1, 0, 0, 0, 659, 661, 1, 0, 0, 0, 660, 658, 1, 0, 0, 0, 661, 662, 5, 96, 0, 0, 662, 680, 1, 0, 0, 0, 663, 665, 3, 138, 69, 0, 664, 666, 5, 67, 0, 0, 665, 664, 1, 0, 0, 0, 665, 666, 1, 0, 0, 0, 666, 667, 1, 0, 0, 0, 667, 668, 5, 73, 0, 0, 668, 669, 5, 95, 0, 0, 669, 674, 3, 164, 82, 0, 670, 671, 5, 58, 0, 0, 671, 673, 3, 164, 82, 0, 672, 670, 1, 0, 0, 0, 673, 676, 1, 0, 0, 0, 674, 672, 1, 0, 0, 0, 674, 675, 1, 0, 0, 0, 675, 677, 1, 0, 0, 0, 676, 674, 1, 0, 0, 0, 677, 678, 5, 96, 0, 0, 678, 680, 1, 0, 0, 0, 679, 633, 1, 0, 0, 0, 679, 640, 1, 0, 0, 0, 679, 647, 1, 0, 0, 0, 679, 663, 1, 0, 0, 0, 680, 135, 1, 0, 0, 0, 681, 684, 3, 48, 24, 0, 682, 683, 5, 56, 0, 0, 683, 685, 3, 10, 5, 0, 684, 682, 1, 0, 0, 0, 684, 685, 1, 0, 0, 0, 685, 686, 1, 0, 0, 0, 686, 687, 5, 57, 0, 0, 687, 688, 3, 154, 77, 0, 688, 137, 1, 0, 0, 0, 689, 695, 3, 140, 70, 0, 690, 691, 3, 140, 70, 0, 691, 692, 3, 166, 83, 0, 692, 693, 3, 140, 70, 0, 693, 695, 1, 0, 0, 0, 694, 689, 1, 0, 0, 0, 694, 690, 1, 0, 0, 0, 695, 139, 1, 0, 0, 0, 696, 697, 6, 70, -1, 0, 697, 701, 3, 142, 71, 0, 698, 699, 7, 5, 0, 0, 699, 701, 3, 140, 70, 3, 700, 696, 1, 0, 0, 0, 700, 698, 1, 0, 0, 0, 701, 710, 1, 0, 0, 0, 702, 703, 10, 2, 0, 0, 703, 704, 7, 6, 0, 0, 704, 709, 3, 140, 70, 3, 705, 706, 10, 1, 0, 0, 706, 707, 7, 5, 0, 0, 707, 709, 3, 140, 70, 2, 708, 702, 1, 0, 0, 0, 708, 705, 1, 0, 0, 0, 709, 712, 1, 0, 0, 0, 710, 708, 1, 0, 0, 0, 710, 711, 1, 0, 0, 0, 711, 141, 1, 0, 0, 0, 712, 710, 1, 0, 0, 0, 713, 714, 6, 71, -1, 0, 714, 722, 3, 154, 77, 0, 715, 722, 3, 48, 24, 0, 716, 722, 3, 144, 72, 0, 717, 718, 5, 95, 0, 0, 718, 719, 3, 132, 66, 0, 719, 720, 5, 96, 0, 0, 720, 722, 1, 0, 0, 0, 721, 713, 1, 0, 0, 0, 721, 715, 1, 0, 0, 0, 721, 716, 1, 0, 0, 0, 721, 717, 1, 0, 0, 0, 722, 728, 1, 0, 0, 0, 723, 724, 10, 1, 0, 0, 724, 725, 5, 56, 0, 0, 725, 727, 3, 10, 5, 0, 726, 723, 1, 0, 0, 0, 727, 730, 1, 0, 0, 0, 728, 726, 1, 0, 0, 0, 728, 729, 1, 0, 0, 0, 729, 143, 1, 0, 0, 0, 730, 728, 1, 0, 0, 0, 731, 732, 3, 146, 73, 0, 732, 746, 5, 95, 0, 0, 733, 747, 5, 85, 0, 0, 734, 739, 3, 132, 66, 0, 735, 736, 5, 58, 0, 0, 736, 738, 3, 132, 66, 0, 737, 735, 1, 0, 0, 0, 738, 741, 1, 0, 0, 0, 739, 737, 1, 0, 0, 0, 739, 740, 1, 0, 0, 0, 740, 744, 1, 0, 0, 0, 741, 739, 1, 0, 0, 0, 742, 743, 5, 58, 0, 0, 743, 745, 3, 148, 74, 0, 744, 742, 1, 0, 0, 0, 744, 745, 1, 0, 0, 0, 745, 747, 1, 0, 0, 0, 746, 733, 1, 0, 0, 0, 746, 734, 1, 0, 0, 0, 746, 747, 1, 0, 0, 0, 747, 748, 1, 0, 0, 0, 748, 749, 5, 96, 0, 0, 749, 145, 1, 0, 0, 0, 750, 754, 3, 66, 33, 0, 751, 754, 5, 62, 0, 0, 752, 754, 5, 65, 0, 0, 753, 750, 1, 0, 0, 0, 753, 751, 1, 0, 0, 0, 753, 752, 1, 0, 0, 0, 754, 147, 1, 0, 0, 0, 755, 764, 5, 88, 0, 0, 756, 761, 3, 150, 75, 0, 757, 758, 5, 58, 0, 0, 758, 760, 3, 150, 75, 0, 759, 757, 1, 0, 0, 0, 760, 763, 1, 0, 0, 0, 761, 759, 1, 0, 0, 0, 761, 762, 1, 0, 0, 0, 762, 765, 1, 0, 0, 0, 763, 761, 1, 0, 0, 0, 764, 756, 1, 0, 0, 0, 764, 765, 1, 0, 0, 0, 765, 766, 1, 0, 0, 0, 766, 767, 5, 89, 0, 0, 767, 149, 1, 0, 0, 0, 768, 769, 3, 164, 82, 0, 769, 770, 5, 57, 0, 0, 770, 771, 3, 152, 76, 0, 771, 151, 1, 0, 0, 0, 772, 775, 3, 154, 77, 0, 773, 775, 3, 148, 74, 0, 774, 772, 1, 0, 0, 0, 774, 773, 1, 0, 0, 0, 775, 153, 1, 0, 0, 0, 776, 819, 5, 68, 0, 0, 777, 778, 3, 162, 81, 0, 778, 779, 5, 97, 0, 0, 779, 819, 1, 0, 0, 0, 780, 819, 3, 160, 80, 0, 781, 819, 3, 162, 81, 0, 782, 819, 3, 156, 78, 0, 783, 819, 3, 62, 31, 0, 784, 819, 3, 164, 82, 0, 785, 786, 5, 93, 0, 0, 786, 791, 3, 158, 79, 0, 787, 788, 5, 58, 0, 0, 788, 790, 3, 158, 79, 0, 789, 787, 1, 0, 0, 0, 790, 793, 1, 0, 0, 0, 791, 789, 1, 0, 0, 0, 791, 792, 1, 0, 0, 0, 792, 794, 1, 0, 0, 0, 793, 791, 1, 0, 0, 0, 794, 795, 5, 94, 0, 0, 795, 819, 1, 0, 0, 0, 796, 797, 5, 93, 0, 0, 797, 802, 3, 156, 78, 0, 798, 799, 5, 58, 0, 0, 799, 801, 3, 156, 78, 0, 800, 798, 1, 0, 0, 0, 801, 804, 1, 0, 0, 0, 802, 800, 1, 0, 0, 0, 802, 803, 1, 0, 0, 0, 803, 805, 1, 0, 0, 0, 804, 802, 1, 0, 0, 0, 805, 806, 5, 94, 0, 0, 806, 819, 1, 0, 0, 0, 807, 808, 5, 93, 0, 0, 808, 813, 3, 164, 82, 0, 809, 810, 5, 58, 0, 0, 810, 812, 3, 164, 82, 0, 811, 809, 1, 0, 0, 0, 812, 815, 1, 0, 0, 0, 813, 811, 1, 0, 0, 0, 813, 814, 1, 0, 0, 0, 814, 816, 1, 0, 0, 0, 815, 813, 1, 0, 0, 0, 816, 817, 5, 94, 0, 0, 817, 819, 1, 0, 0, 0, 818, 776, 1, 0, 0, 0, 818, 777, 1, 0, 0, 0, 818, 780, 1, 0, 0, 0, 818, 781, 1, 0, 0, 0, 818, 782, 1, 0, 0, 0, 818, 783, 1, 0, 0, 0, 818, 784, 1, 0, 0, 0, 818, 785, 1, 0, 0, 0, 818, 796, 1, 0, 0, 0, 818, 807, 1, 0, 0, 0, 819, 155, 1, 0, 0, 0, 820, 821, 7, 7, 0, 0, 821, 157, 1, 0, 0, 0, 822, 825, 3, 160, 80, 0, 823, 825, 3, 162, 81, 0, 824, 822, 1, 0, 0, 0, 824, 823, 1, 0, 0, 0, 825, 159, 1, 0, 0, 0, 826, 828, 7, 5, 0, 0, 827, 826, 1, 0, 0, 0, 827, 828, 1, 0, 0, 0, 828, 829, 1, 0, 0, 0, 829, 830, 5, 51, 0, 0, 830, 161, 1, 0, 0, 0, 831, 833, 7, 5, 0, 0, 832, 831, 1, 0, 0, 0, 832, 833, 1, 0, 0, 0, 833, 834, 1, 0, 0, 0, 834, 835, 5, 50, 0, 0, 835, 163, 1, 0, 0, 0, 836, 837, 5, 49, 0, 0, 837, 165, 1, 0, 0, 0, 838, 839, 7, 8, 0, 0, 839, 167, 1, 0, 0, 0, 840, 841, 7, 9, 0, 0, 841, 842, 5, 110, 0, 0, 842, 843, 3, 170, 85, 0, 843, 844, 3, 172, 86, 0, 844, 169, 1, 0, 0, 0, 845, 846, 4, 85, 15, 0, 846, 848, 3, 28, 14, 0, 847, 849, 5, 128, 0, 0, 848, 847, 1, 0, 0, 0, 848, 849, 1, 0, 0, 0, 849, 850, 1, 0, 0, 0, 850, 851, 5, 103, 0, 0, 851, 854, 1, 0, 0, 0, 852, 854, 3, 28, 14, 0, 853, 845, 1, 0, 0, 0, 853, 852, 1, 0, 0, 0, 854, 171, 1, 0, 0, 0, 855, 856, 5, 70, 0, 0, 856, 861, 3, 174, 87, 0, 857, 858, 5, 58, 0, 0, 858, 860, 3, 174, 87, 0, 859, 857, 1, 0, 0, 0, 860, 863, 1, 0, 0, 0, 861, 859, 1, 0, 0, 0, 861, 862, 1, 0, 0, 0, 862, 173, 1, 0, 0, 0, 863, 861, 1, 0, 0, 0, 864, 865, 3, 138, 69, 0, 865, 175, 1, 0, 0, 0, 82, 187, 197, 225, 240, 246, 255, 261, 274, 278, 289, 305, 313, 317, 324, 330, 335, 344, 351, 357, 366, 373, 381, 389, 393, 397, 402, 413, 418, 422, 436, 447, 453, 460, 469, 492, 500, 503, 510, 521, 528, 536, 550, 559, 570, 584, 598, 607, 615, 620, 628, 630, 635, 642, 649, 658, 665, 674, 679, 684, 694, 700, 708, 710, 721, 728, 739, 744, 746, 753, 761, 764, 774, 791, 802, 813, 818, 824, 827, 832, 848, 853, 861]
\ No newline at end of file
+[4, 1, 135, 877, 2, 0, 7, 0, 2, 1, 7, 1, 2, 2, 7, 2, 2, 3, 7, 3, 2, 4, 7, 4, 2, 5, 7, 5, 2, 6, 7, 6, 2, 7, 7, 7, 2, 8, 7, 8, 2, 9, 7, 9, 2, 10, 7, 10, 2, 11, 7, 11, 2, 12, 7, 12, 2, 13, 7, 13, 2, 14, 7, 14, 2, 15, 7, 15, 2, 16, 7, 16, 2, 17, 7, 17, 2, 18, 7, 18, 2, 19, 7, 19, 2, 20, 7, 20, 2, 21, 7, 21, 2, 22, 7, 22, 2, 23, 7, 23, 2, 24, 7, 24, 2, 25, 7, 25, 2, 26, 7, 26, 2, 27, 7, 27, 2, 28, 7, 28, 2, 29, 7, 29, 2, 30, 7, 30, 2, 31, 7, 31, 2, 32, 7, 32, 2, 33, 7, 33, 2, 34, 7, 34, 2, 35, 7, 35, 2, 36, 7, 36, 2, 37, 7, 37, 2, 38, 7, 38, 2, 39, 7, 39, 2, 40, 7, 40, 2, 41, 7, 41, 2, 42, 7, 42, 2, 43, 7, 43, 2, 44, 7, 44, 2, 45, 7, 45, 2, 46, 7, 46, 2, 47, 7, 47, 2, 48, 7, 48, 2, 49, 7, 49, 2, 50, 7, 50, 2, 51, 7, 51, 2, 52, 7, 52, 2, 53, 7, 53, 2, 54, 7, 54, 2, 55, 7, 55, 2, 56, 7, 56, 2, 57, 7, 57, 2, 58, 7, 58, 2, 59, 7, 59, 2, 60, 7, 60, 2, 61, 7, 61, 2, 62, 7, 62, 2, 63, 7, 63, 2, 64, 7, 64, 2, 65, 7, 65, 2, 66, 7, 66, 2, 67, 7, 67, 2, 68, 7, 68, 2, 69, 7, 69, 2, 70, 7, 70, 2, 71, 7, 71, 2, 72, 7, 72, 2, 73, 7, 73, 2, 74, 7, 74, 2, 75, 7, 75, 2, 76, 7, 76, 2, 77, 7, 77, 2, 78, 7, 78, 2, 79, 7, 79, 2, 80, 7, 80, 2, 81, 7, 81, 2, 82, 7, 82, 2, 83, 7, 83, 2, 84, 7, 84, 2, 85, 7, 85, 2, 86, 7, 86, 2, 87, 7, 87, 1, 0, 1, 0, 1, 0, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 5, 1, 186, 8, 1, 10, 1, 12, 1, 189, 9, 1, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 1, 2, 3, 2, 198, 8, 2, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 1, 3, 3, 3, 226, 8, 3, 1, 4, 1, 4, 1, 4, 1, 5, 1, 5, 1, 6, 1, 6, 1, 6, 1, 7, 1, 7, 1, 7, 5, 7, 239, 8, 7, 10, 7, 12, 7, 242, 9, 7, 1, 8, 1, 8, 1, 8, 3, 8, 247, 8, 8, 1, 8, 1, 8, 1, 9, 1, 9, 1, 9, 5, 9, 254, 8, 9, 10, 9, 12, 9, 257, 9, 9, 1, 10, 1, 10, 1, 10, 3, 10, 262, 8, 10, 1, 11, 1, 11, 1, 11, 1, 12, 1, 12, 1, 12, 1, 13, 1, 13, 1, 13, 5, 13, 273, 8, 13, 10, 13, 12, 13, 276, 9, 13, 1, 13, 3, 13, 279, 8, 13, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 1, 14, 3, 14, 290, 8, 14, 1, 15, 1, 15, 1, 16, 1, 16, 1, 17, 1, 17, 1, 18, 1, 18, 1, 19, 1, 19, 1, 19, 1, 19, 5, 19, 304, 8, 19, 10, 19, 12, 19, 307, 9, 19, 1, 20, 1, 20, 1, 20, 1, 21, 1, 21, 3, 21, 314, 8, 21, 1, 21, 1, 21, 3, 21, 318, 8, 21, 1, 22, 1, 22, 1, 22, 5, 22, 323, 8, 22, 10, 22, 12, 22, 326, 9, 22, 1, 23, 1, 23, 1, 23, 3, 23, 331, 8, 23, 1, 24, 1, 24, 1, 24, 3, 24, 336, 8, 24, 1, 24, 1, 24, 1, 24, 1, 24, 1, 24, 1, 24, 1, 24, 3, 24, 345, 8, 24, 1, 25, 1, 25, 1, 25, 5, 25, 350, 8, 25, 10, 25, 12, 25, 353, 9, 25, 1, 26, 1, 26, 1, 26, 3, 26, 358, 8, 26, 1, 26, 1, 26, 1, 26, 1, 26, 1, 26, 1, 26, 1, 26, 3, 26, 367, 8, 26, 1, 27, 1, 27, 1, 27, 5, 27, 372, 8, 27, 10, 27, 12, 27, 375, 9, 27, 1, 28, 1, 28, 1, 28, 5, 28, 380, 8, 28, 10, 28, 12, 28, 383, 9, 28, 1, 29, 1, 29, 1, 30, 1, 30, 1, 30, 3, 30, 390, 8, 30, 1, 31, 1, 31, 3, 31, 394, 8, 31, 1, 32, 1, 32, 3, 32, 398, 8, 32, 1, 33, 1, 33, 1, 33, 3, 33, 403, 8, 33, 1, 34, 1, 34, 1, 34, 1, 35, 1, 35, 1, 35, 1, 35, 5, 35, 412, 8, 35, 10, 35, 12, 35, 415, 9, 35, 1, 36, 1, 36, 3, 36, 419, 8, 36, 1, 36, 1, 36, 3, 36, 423, 8, 36, 1, 37, 1, 37, 1, 37, 1, 38, 1, 38, 1, 38, 1, 39, 1, 39, 1, 39, 1, 39, 5, 39, 435, 8, 39, 10, 39, 12, 39, 438, 9, 39, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 1, 40, 3, 40, 448, 8, 40, 1, 41, 1, 41, 1, 41, 1, 41, 3, 41, 454, 8, 41, 1, 42, 1, 42, 1, 42, 5, 42, 459, 8, 42, 10, 42, 12, 42, 462, 9, 42, 1, 43, 1, 43, 1, 43, 1, 43, 1, 44, 1, 44, 3, 44, 470, 8, 44, 1, 45, 1, 45, 1, 45, 1, 45, 1, 46, 1, 46, 1, 46, 1, 47, 1, 47, 1, 47, 1, 48, 1, 48, 1, 48, 1, 48, 1, 49, 1, 49, 1, 49, 1, 50, 1, 50, 1, 50, 1, 50, 3, 50, 493, 8, 50, 1, 50, 1, 50, 1, 50, 1, 50, 5, 50, 499, 8, 50, 10, 50, 12, 50, 502, 9, 50, 3, 50, 504, 8, 50, 1, 51, 1, 51, 1, 52, 1, 52, 1, 52, 3, 52, 511, 8, 52, 1, 52, 1, 52, 1, 53, 1, 53, 1, 53, 1, 54, 1, 54, 1, 54, 1, 54, 3, 54, 522, 8, 54, 1, 54, 1, 54, 1, 54, 1, 54, 1, 54, 3, 54, 529, 8, 54, 1, 55, 1, 55, 1, 55, 1, 56, 4, 56, 535, 8, 56, 11, 56, 12, 56, 536, 1, 57, 1, 57, 1, 57, 1, 57, 1, 58, 1, 58, 1, 58, 1, 58, 1, 58, 1, 58, 5, 58, 549, 8, 58, 10, 58, 12, 58, 552, 9, 58, 1, 59, 1, 59, 1, 60, 1, 60, 1, 60, 1, 60, 3, 60, 560, 8, 60, 1, 60, 1, 60, 1, 60, 1, 60, 1, 60, 1, 61, 1, 61, 1, 61, 1, 61, 3, 61, 571, 8, 61, 1, 61, 1, 61, 1, 61, 1, 62, 1, 62, 1, 62, 1, 62, 1, 62, 1, 63, 1, 63, 1, 63, 1, 63, 3, 63, 585, 8, 63, 1, 64, 1, 64, 1, 64, 1, 65, 1, 65, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 3, 66, 599, 8, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 5, 66, 606, 8, 66, 10, 66, 12, 66, 609, 9, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 3, 66, 616, 8, 66, 1, 66, 1, 66, 1, 66, 3, 66, 621, 8, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 1, 66, 5, 66, 629, 8, 66, 10, 66, 12, 66, 632, 9, 66, 1, 67, 1, 67, 3, 67, 636, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 643, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 650, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 5, 67, 657, 8, 67, 10, 67, 12, 67, 660, 9, 67, 1, 67, 1, 67, 1, 67, 1, 67, 3, 67, 666, 8, 67, 1, 67, 1, 67, 1, 67, 1, 67, 1, 67, 5, 67, 673, 8, 67, 10, 67, 12, 67, 676, 9, 67, 1, 67, 1, 67, 3, 67, 680, 8, 67, 1, 68, 1, 68, 1, 68, 3, 68, 685, 8, 68, 1, 68, 1, 68, 1, 68, 1, 69, 1, 69, 3, 69, 692, 8, 69, 1, 70, 1, 70, 1, 70, 1, 70, 1, 71, 1, 71, 1, 71, 1, 71, 3, 71, 702, 8, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 1, 71, 5, 71, 710, 8, 71, 10, 71, 12, 71, 713, 9, 71, 1, 72, 1, 72, 1, 72, 1, 72, 1, 72, 1, 72, 1, 72, 1, 72, 3, 72, 723, 8, 72, 1, 72, 1, 72, 1, 72, 5, 72, 728, 8, 72, 10, 72, 12, 72, 731, 9, 72, 1, 73, 1, 73, 1, 73, 1, 73, 1, 73, 1, 73, 5, 73, 739, 8, 73, 10, 73, 12, 73, 742, 9, 73, 1, 73, 1, 73, 3, 73, 746, 8, 73, 3, 73, 748, 8, 73, 1, 73, 1, 73, 1, 74, 1, 74, 1, 74, 3, 74, 755, 8, 74, 1, 75, 1, 75, 1, 75, 1, 75, 5, 75, 761, 8, 75, 10, 75, 12, 75, 764, 9, 75, 3, 75, 766, 8, 75, 1, 75, 1, 75, 1, 76, 1, 76, 1, 76, 1, 76, 1, 77, 1, 77, 3, 77, 776, 8, 77, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 5, 78, 791, 8, 78, 10, 78, 12, 78, 794, 9, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 5, 78, 802, 8, 78, 10, 78, 12, 78, 805, 9, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 1, 78, 5, 78, 813, 8, 78, 10, 78, 12, 78, 816, 9, 78, 1, 78, 1, 78, 3, 78, 820, 8, 78, 1, 79, 1, 79, 1, 80, 1, 80, 3, 80, 826, 8, 80, 1, 81, 3, 81, 829, 8, 81, 1, 81, 1, 81, 1, 82, 3, 82, 834, 8, 82, 1, 82, 1, 82, 1, 83, 1, 83, 1, 84, 1, 84, 1, 85, 1, 85, 1, 85, 1, 85, 1, 85, 1, 86, 1, 86, 1, 86, 3, 86, 850, 8, 86, 1, 86, 1, 86, 1, 86, 3, 86, 855, 8, 86, 1, 87, 1, 87, 1, 87, 1, 87, 5, 87, 861, 8, 87, 10, 87, 12, 87, 864, 9, 87, 1, 87, 1, 87, 1, 87, 1, 87, 5, 87, 870, 8, 87, 10, 87, 12, 87, 873, 9, 87, 3, 87, 875, 8, 87, 1, 87, 0, 5, 2, 116, 132, 142, 144, 88, 0, 2, 4, 6, 8, 10, 12, 14, 16, 18, 20, 22, 24, 26, 28, 30, 32, 34, 36, 38, 40, 42, 44, 46, 48, 50, 52, 54, 56, 58, 60, 62, 64, 66, 68, 70, 72, 74, 76, 78, 80, 82, 84, 86, 88, 90, 92, 94, 96, 98, 100, 102, 104, 106, 108, 110, 112, 114, 116, 118, 120, 122, 124, 126, 128, 130, 132, 134, 136, 138, 140, 142, 144, 146, 148, 150, 152, 154, 156, 158, 160, 162, 164, 166, 168, 170, 172, 174, 0, 10, 2, 0, 49, 49, 103, 103, 1, 0, 97, 98, 2, 0, 53, 53, 59, 59, 2, 0, 62, 62, 65, 65, 2, 0, 38, 38, 49, 49, 1, 0, 83, 84, 1, 0, 85, 87, 2, 0, 61, 61, 74, 74, 2, 0, 76, 76, 78, 82, 2, 0, 23, 23, 25, 26, 916, 0, 176, 1, 0, 0, 0, 2, 179, 1, 0, 0, 0, 4, 197, 1, 0, 0, 0, 6, 225, 1, 0, 0, 0, 8, 227, 1, 0, 0, 0, 10, 230, 1, 0, 0, 0, 12, 232, 1, 0, 0, 0, 14, 235, 1, 0, 0, 0, 16, 246, 1, 0, 0, 0, 18, 250, 1, 0, 0, 0, 20, 258, 1, 0, 0, 0, 22, 263, 1, 0, 0, 0, 24, 266, 1, 0, 0, 0, 26, 269, 1, 0, 0, 0, 28, 289, 1, 0, 0, 0, 30, 291, 1, 0, 0, 0, 32, 293, 1, 0, 0, 0, 34, 295, 1, 0, 0, 0, 36, 297, 1, 0, 0, 0, 38, 299, 1, 0, 0, 0, 40, 308, 1, 0, 0, 0, 42, 311, 1, 0, 0, 0, 44, 319, 1, 0, 0, 0, 46, 327, 1, 0, 0, 0, 48, 344, 1, 0, 0, 0, 50, 346, 1, 0, 0, 0, 52, 366, 1, 0, 0, 0, 54, 368, 1, 0, 0, 0, 56, 376, 1, 0, 0, 0, 58, 384, 1, 0, 0, 0, 60, 389, 1, 0, 0, 0, 62, 393, 1, 0, 0, 0, 64, 397, 1, 0, 0, 0, 66, 402, 1, 0, 0, 0, 68, 404, 1, 0, 0, 0, 70, 407, 1, 0, 0, 0, 72, 416, 1, 0, 0, 0, 74, 424, 1, 0, 0, 0, 76, 427, 1, 0, 0, 0, 78, 430, 1, 0, 0, 0, 80, 447, 1, 0, 0, 0, 82, 449, 1, 0, 0, 0, 84, 455, 1, 0, 0, 0, 86, 463, 1, 0, 0, 0, 88, 469, 1, 0, 0, 0, 90, 471, 1, 0, 0, 0, 92, 475, 1, 0, 0, 0, 94, 478, 1, 0, 0, 0, 96, 481, 1, 0, 0, 0, 98, 485, 1, 0, 0, 0, 100, 488, 1, 0, 0, 0, 102, 505, 1, 0, 0, 0, 104, 510, 1, 0, 0, 0, 106, 514, 1, 0, 0, 0, 108, 517, 1, 0, 0, 0, 110, 530, 1, 0, 0, 0, 112, 534, 1, 0, 0, 0, 114, 538, 1, 0, 0, 0, 116, 542, 1, 0, 0, 0, 118, 553, 1, 0, 0, 0, 120, 555, 1, 0, 0, 0, 122, 566, 1, 0, 0, 0, 124, 575, 1, 0, 0, 0, 126, 580, 1, 0, 0, 0, 128, 586, 1, 0, 0, 0, 130, 589, 1, 0, 0, 0, 132, 620, 1, 0, 0, 0, 134, 679, 1, 0, 0, 0, 136, 681, 1, 0, 0, 0, 138, 691, 1, 0, 0, 0, 140, 693, 1, 0, 0, 0, 142, 701, 1, 0, 0, 0, 144, 722, 1, 0, 0, 0, 146, 732, 1, 0, 0, 0, 148, 754, 1, 0, 0, 0, 150, 756, 1, 0, 0, 0, 152, 769, 1, 0, 0, 0, 154, 775, 1, 0, 0, 0, 156, 819, 1, 0, 0, 0, 158, 821, 1, 0, 0, 0, 160, 825, 1, 0, 0, 0, 162, 828, 1, 0, 0, 0, 164, 833, 1, 0, 0, 0, 166, 837, 1, 0, 0, 0, 168, 839, 1, 0, 0, 0, 170, 841, 1, 0, 0, 0, 172, 854, 1, 0, 0, 0, 174, 874, 1, 0, 0, 0, 176, 177, 3, 2, 1, 0, 177, 178, 5, 0, 0, 1, 178, 1, 1, 0, 0, 0, 179, 180, 6, 1, -1, 0, 180, 181, 3, 4, 2, 0, 181, 187, 1, 0, 0, 0, 182, 183, 10, 1, 0, 0, 183, 184, 5, 48, 0, 0, 184, 186, 3, 6, 3, 0, 185, 182, 1, 0, 0, 0, 186, 189, 1, 0, 0, 0, 187, 185, 1, 0, 0, 0, 187, 188, 1, 0, 0, 0, 188, 3, 1, 0, 0, 0, 189, 187, 1, 0, 0, 0, 190, 198, 3, 22, 11, 0, 191, 198, 3, 12, 6, 0, 192, 198, 3, 98, 49, 0, 193, 194, 4, 2, 1, 0, 194, 198, 3, 24, 12, 0, 195, 196, 4, 2, 2, 0, 196, 198, 3, 94, 47, 0, 197, 190, 1, 0, 0, 0, 197, 191, 1, 0, 0, 0, 197, 192, 1, 0, 0, 0, 197, 193, 1, 0, 0, 0, 197, 195, 1, 0, 0, 0, 198, 5, 1, 0, 0, 0, 199, 226, 3, 40, 20, 0, 200, 226, 3, 8, 4, 0, 201, 226, 3, 74, 37, 0, 202, 226, 3, 68, 34, 0, 203, 226, 3, 42, 21, 0, 204, 226, 3, 70, 35, 0, 205, 226, 3, 76, 38, 0, 206, 226, 3, 78, 39, 0, 207, 226, 3, 82, 41, 0, 208, 226, 3, 90, 45, 0, 209, 226, 3, 100, 50, 0, 210, 226, 3, 92, 46, 0, 211, 226, 3, 170, 85, 0, 212, 226, 3, 108, 54, 0, 213, 226, 3, 122, 61, 0, 214, 226, 3, 106, 53, 0, 215, 226, 3, 110, 55, 0, 216, 226, 3, 120, 60, 0, 217, 218, 4, 3, 3, 0, 218, 226, 3, 126, 63, 0, 219, 220, 4, 3, 4, 0, 220, 226, 3, 124, 62, 0, 221, 222, 4, 3, 5, 0, 222, 226, 3, 128, 64, 0, 223, 224, 4, 3, 6, 0, 224, 226, 3, 130, 65, 0, 225, 199, 1, 0, 0, 0, 225, 200, 1, 0, 0, 0, 225, 201, 1, 0, 0, 0, 225, 202, 1, 0, 0, 0, 225, 203, 1, 0, 0, 0, 225, 204, 1, 0, 0, 0, 225, 205, 1, 0, 0, 0, 225, 206, 1, 0, 0, 0, 225, 207, 1, 0, 0, 0, 225, 208, 1, 0, 0, 0, 225, 209, 1, 0, 0, 0, 225, 210, 1, 0, 0, 0, 225, 211, 1, 0, 0, 0, 225, 212, 1, 0, 0, 0, 225, 213, 1, 0, 0, 0, 225, 214, 1, 0, 0, 0, 225, 215, 1, 0, 0, 0, 225, 216, 1, 0, 0, 0, 225, 217, 1, 0, 0, 0, 225, 219, 1, 0, 0, 0, 225, 221, 1, 0, 0, 0, 225, 223, 1, 0, 0, 0, 226, 7, 1, 0, 0, 0, 227, 228, 5, 17, 0, 0, 228, 229, 3, 132, 66, 0, 229, 9, 1, 0, 0, 0, 230, 231, 3, 58, 29, 0, 231, 11, 1, 0, 0, 0, 232, 233, 5, 13, 0, 0, 233, 234, 3, 14, 7, 0, 234, 13, 1, 0, 0, 0, 235, 240, 3, 16, 8, 0, 236, 237, 5, 58, 0, 0, 237, 239, 3, 16, 8, 0, 238, 236, 1, 0, 0, 0, 239, 242, 1, 0, 0, 0, 240, 238, 1, 0, 0, 0, 240, 241, 1, 0, 0, 0, 241, 15, 1, 0, 0, 0, 242, 240, 1, 0, 0, 0, 243, 244, 3, 48, 24, 0, 244, 245, 5, 54, 0, 0, 245, 247, 1, 0, 0, 0, 246, 243, 1, 0, 0, 0, 246, 247, 1, 0, 0, 0, 247, 248, 1, 0, 0, 0, 248, 249, 3, 132, 66, 0, 249, 17, 1, 0, 0, 0, 250, 255, 3, 20, 10, 0, 251, 252, 5, 58, 0, 0, 252, 254, 3, 20, 10, 0, 253, 251, 1, 0, 0, 0, 254, 257, 1, 0, 0, 0, 255, 253, 1, 0, 0, 0, 255, 256, 1, 0, 0, 0, 256, 19, 1, 0, 0, 0, 257, 255, 1, 0, 0, 0, 258, 261, 3, 48, 24, 0, 259, 260, 5, 54, 0, 0, 260, 262, 3, 132, 66, 0, 261, 259, 1, 0, 0, 0, 261, 262, 1, 0, 0, 0, 262, 21, 1, 0, 0, 0, 263, 264, 5, 19, 0, 0, 264, 265, 3, 26, 13, 0, 265, 23, 1, 0, 0, 0, 266, 267, 5, 20, 0, 0, 267, 268, 3, 26, 13, 0, 268, 25, 1, 0, 0, 0, 269, 274, 3, 28, 14, 0, 270, 271, 5, 58, 0, 0, 271, 273, 3, 28, 14, 0, 272, 270, 1, 0, 0, 0, 273, 276, 1, 0, 0, 0, 274, 272, 1, 0, 0, 0, 274, 275, 1, 0, 0, 0, 275, 278, 1, 0, 0, 0, 276, 274, 1, 0, 0, 0, 277, 279, 3, 38, 19, 0, 278, 277, 1, 0, 0, 0, 278, 279, 1, 0, 0, 0, 279, 27, 1, 0, 0, 0, 280, 281, 3, 30, 15, 0, 281, 282, 5, 57, 0, 0, 282, 283, 3, 34, 17, 0, 283, 290, 1, 0, 0, 0, 284, 285, 3, 34, 17, 0, 285, 286, 5, 56, 0, 0, 286, 287, 3, 32, 16, 0, 287, 290, 1, 0, 0, 0, 288, 290, 3, 36, 18, 0, 289, 280, 1, 0, 0, 0, 289, 284, 1, 0, 0, 0, 289, 288, 1, 0, 0, 0, 290, 29, 1, 0, 0, 0, 291, 292, 5, 103, 0, 0, 292, 31, 1, 0, 0, 0, 293, 294, 5, 103, 0, 0, 294, 33, 1, 0, 0, 0, 295, 296, 5, 103, 0, 0, 296, 35, 1, 0, 0, 0, 297, 298, 7, 0, 0, 0, 298, 37, 1, 0, 0, 0, 299, 300, 5, 102, 0, 0, 300, 305, 5, 103, 0, 0, 301, 302, 5, 58, 0, 0, 302, 304, 5, 103, 0, 0, 303, 301, 1, 0, 0, 0, 304, 307, 1, 0, 0, 0, 305, 303, 1, 0, 0, 0, 305, 306, 1, 0, 0, 0, 306, 39, 1, 0, 0, 0, 307, 305, 1, 0, 0, 0, 308, 309, 5, 9, 0, 0, 309, 310, 3, 14, 7, 0, 310, 41, 1, 0, 0, 0, 311, 313, 5, 16, 0, 0, 312, 314, 3, 44, 22, 0, 313, 312, 1, 0, 0, 0, 313, 314, 1, 0, 0, 0, 314, 317, 1, 0, 0, 0, 315, 316, 5, 55, 0, 0, 316, 318, 3, 14, 7, 0, 317, 315, 1, 0, 0, 0, 317, 318, 1, 0, 0, 0, 318, 43, 1, 0, 0, 0, 319, 324, 3, 46, 23, 0, 320, 321, 5, 58, 0, 0, 321, 323, 3, 46, 23, 0, 322, 320, 1, 0, 0, 0, 323, 326, 1, 0, 0, 0, 324, 322, 1, 0, 0, 0, 324, 325, 1, 0, 0, 0, 325, 45, 1, 0, 0, 0, 326, 324, 1, 0, 0, 0, 327, 330, 3, 16, 8, 0, 328, 329, 5, 17, 0, 0, 329, 331, 3, 132, 66, 0, 330, 328, 1, 0, 0, 0, 330, 331, 1, 0, 0, 0, 331, 47, 1, 0, 0, 0, 332, 333, 4, 24, 7, 0, 333, 335, 5, 93, 0, 0, 334, 336, 5, 97, 0, 0, 335, 334, 1, 0, 0, 0, 335, 336, 1, 0, 0, 0, 336, 337, 1, 0, 0, 0, 337, 338, 5, 94, 0, 0, 338, 339, 5, 60, 0, 0, 339, 340, 5, 93, 0, 0, 340, 341, 3, 50, 25, 0, 341, 342, 5, 94, 0, 0, 342, 345, 1, 0, 0, 0, 343, 345, 3, 50, 25, 0, 344, 332, 1, 0, 0, 0, 344, 343, 1, 0, 0, 0, 345, 49, 1, 0, 0, 0, 346, 351, 3, 66, 33, 0, 347, 348, 5, 60, 0, 0, 348, 350, 3, 66, 33, 0, 349, 347, 1, 0, 0, 0, 350, 353, 1, 0, 0, 0, 351, 349, 1, 0, 0, 0, 351, 352, 1, 0, 0, 0, 352, 51, 1, 0, 0, 0, 353, 351, 1, 0, 0, 0, 354, 355, 4, 26, 8, 0, 355, 357, 5, 93, 0, 0, 356, 358, 5, 124, 0, 0, 357, 356, 1, 0, 0, 0, 357, 358, 1, 0, 0, 0, 358, 359, 1, 0, 0, 0, 359, 360, 5, 94, 0, 0, 360, 361, 5, 60, 0, 0, 361, 362, 5, 93, 0, 0, 362, 363, 3, 54, 27, 0, 363, 364, 5, 94, 0, 0, 364, 367, 1, 0, 0, 0, 365, 367, 3, 54, 27, 0, 366, 354, 1, 0, 0, 0, 366, 365, 1, 0, 0, 0, 367, 53, 1, 0, 0, 0, 368, 373, 3, 60, 30, 0, 369, 370, 5, 60, 0, 0, 370, 372, 3, 60, 30, 0, 371, 369, 1, 0, 0, 0, 372, 375, 1, 0, 0, 0, 373, 371, 1, 0, 0, 0, 373, 374, 1, 0, 0, 0, 374, 55, 1, 0, 0, 0, 375, 373, 1, 0, 0, 0, 376, 381, 3, 52, 26, 0, 377, 378, 5, 58, 0, 0, 378, 380, 3, 52, 26, 0, 379, 377, 1, 0, 0, 0, 380, 383, 1, 0, 0, 0, 381, 379, 1, 0, 0, 0, 381, 382, 1, 0, 0, 0, 382, 57, 1, 0, 0, 0, 383, 381, 1, 0, 0, 0, 384, 385, 7, 1, 0, 0, 385, 59, 1, 0, 0, 0, 386, 390, 5, 124, 0, 0, 387, 390, 3, 62, 31, 0, 388, 390, 3, 64, 32, 0, 389, 386, 1, 0, 0, 0, 389, 387, 1, 0, 0, 0, 389, 388, 1, 0, 0, 0, 390, 61, 1, 0, 0, 0, 391, 394, 5, 72, 0, 0, 392, 394, 5, 91, 0, 0, 393, 391, 1, 0, 0, 0, 393, 392, 1, 0, 0, 0, 394, 63, 1, 0, 0, 0, 395, 398, 5, 90, 0, 0, 396, 398, 5, 92, 0, 0, 397, 395, 1, 0, 0, 0, 397, 396, 1, 0, 0, 0, 398, 65, 1, 0, 0, 0, 399, 403, 3, 58, 29, 0, 400, 403, 3, 62, 31, 0, 401, 403, 3, 64, 32, 0, 402, 399, 1, 0, 0, 0, 402, 400, 1, 0, 0, 0, 402, 401, 1, 0, 0, 0, 403, 67, 1, 0, 0, 0, 404, 405, 5, 11, 0, 0, 405, 406, 3, 156, 78, 0, 406, 69, 1, 0, 0, 0, 407, 408, 5, 15, 0, 0, 408, 413, 3, 72, 36, 0, 409, 410, 5, 58, 0, 0, 410, 412, 3, 72, 36, 0, 411, 409, 1, 0, 0, 0, 412, 415, 1, 0, 0, 0, 413, 411, 1, 0, 0, 0, 413, 414, 1, 0, 0, 0, 414, 71, 1, 0, 0, 0, 415, 413, 1, 0, 0, 0, 416, 418, 3, 132, 66, 0, 417, 419, 7, 2, 0, 0, 418, 417, 1, 0, 0, 0, 418, 419, 1, 0, 0, 0, 419, 422, 1, 0, 0, 0, 420, 421, 5, 69, 0, 0, 421, 423, 7, 3, 0, 0, 422, 420, 1, 0, 0, 0, 422, 423, 1, 0, 0, 0, 423, 73, 1, 0, 0, 0, 424, 425, 5, 30, 0, 0, 425, 426, 3, 56, 28, 0, 426, 75, 1, 0, 0, 0, 427, 428, 5, 29, 0, 0, 428, 429, 3, 56, 28, 0, 429, 77, 1, 0, 0, 0, 430, 431, 5, 32, 0, 0, 431, 436, 3, 80, 40, 0, 432, 433, 5, 58, 0, 0, 433, 435, 3, 80, 40, 0, 434, 432, 1, 0, 0, 0, 435, 438, 1, 0, 0, 0, 436, 434, 1, 0, 0, 0, 436, 437, 1, 0, 0, 0, 437, 79, 1, 0, 0, 0, 438, 436, 1, 0, 0, 0, 439, 440, 3, 52, 26, 0, 440, 441, 5, 128, 0, 0, 441, 442, 3, 52, 26, 0, 442, 448, 1, 0, 0, 0, 443, 444, 3, 52, 26, 0, 444, 445, 5, 54, 0, 0, 445, 446, 3, 52, 26, 0, 446, 448, 1, 0, 0, 0, 447, 439, 1, 0, 0, 0, 447, 443, 1, 0, 0, 0, 448, 81, 1, 0, 0, 0, 449, 450, 5, 8, 0, 0, 450, 451, 3, 144, 72, 0, 451, 453, 3, 166, 83, 0, 452, 454, 3, 84, 42, 0, 453, 452, 1, 0, 0, 0, 453, 454, 1, 0, 0, 0, 454, 83, 1, 0, 0, 0, 455, 460, 3, 86, 43, 0, 456, 457, 5, 58, 0, 0, 457, 459, 3, 86, 43, 0, 458, 456, 1, 0, 0, 0, 459, 462, 1, 0, 0, 0, 460, 458, 1, 0, 0, 0, 460, 461, 1, 0, 0, 0, 461, 85, 1, 0, 0, 0, 462, 460, 1, 0, 0, 0, 463, 464, 3, 58, 29, 0, 464, 465, 5, 54, 0, 0, 465, 466, 3, 156, 78, 0, 466, 87, 1, 0, 0, 0, 467, 468, 5, 75, 0, 0, 468, 470, 3, 150, 75, 0, 469, 467, 1, 0, 0, 0, 469, 470, 1, 0, 0, 0, 470, 89, 1, 0, 0, 0, 471, 472, 5, 10, 0, 0, 472, 473, 3, 144, 72, 0, 473, 474, 3, 166, 83, 0, 474, 91, 1, 0, 0, 0, 475, 476, 5, 28, 0, 0, 476, 477, 3, 48, 24, 0, 477, 93, 1, 0, 0, 0, 478, 479, 5, 6, 0, 0, 479, 480, 3, 96, 48, 0, 480, 95, 1, 0, 0, 0, 481, 482, 5, 95, 0, 0, 482, 483, 3, 2, 1, 0, 483, 484, 5, 96, 0, 0, 484, 97, 1, 0, 0, 0, 485, 486, 5, 33, 0, 0, 486, 487, 5, 132, 0, 0, 487, 99, 1, 0, 0, 0, 488, 489, 5, 5, 0, 0, 489, 492, 3, 102, 51, 0, 490, 491, 5, 70, 0, 0, 491, 493, 3, 52, 26, 0, 492, 490, 1, 0, 0, 0, 492, 493, 1, 0, 0, 0, 493, 503, 1, 0, 0, 0, 494, 495, 5, 75, 0, 0, 495, 500, 3, 104, 52, 0, 496, 497, 5, 58, 0, 0, 497, 499, 3, 104, 52, 0, 498, 496, 1, 0, 0, 0, 499, 502, 1, 0, 0, 0, 500, 498, 1, 0, 0, 0, 500, 501, 1, 0, 0, 0, 501, 504, 1, 0, 0, 0, 502, 500, 1, 0, 0, 0, 503, 494, 1, 0, 0, 0, 503, 504, 1, 0, 0, 0, 504, 101, 1, 0, 0, 0, 505, 506, 7, 4, 0, 0, 506, 103, 1, 0, 0, 0, 507, 508, 3, 52, 26, 0, 508, 509, 5, 54, 0, 0, 509, 511, 1, 0, 0, 0, 510, 507, 1, 0, 0, 0, 510, 511, 1, 0, 0, 0, 511, 512, 1, 0, 0, 0, 512, 513, 3, 52, 26, 0, 513, 105, 1, 0, 0, 0, 514, 515, 5, 14, 0, 0, 515, 516, 3, 156, 78, 0, 516, 107, 1, 0, 0, 0, 517, 518, 5, 4, 0, 0, 518, 521, 3, 48, 24, 0, 519, 520, 5, 70, 0, 0, 520, 522, 3, 48, 24, 0, 521, 519, 1, 0, 0, 0, 521, 522, 1, 0, 0, 0, 522, 528, 1, 0, 0, 0, 523, 524, 5, 128, 0, 0, 524, 525, 3, 48, 24, 0, 525, 526, 5, 58, 0, 0, 526, 527, 3, 48, 24, 0, 527, 529, 1, 0, 0, 0, 528, 523, 1, 0, 0, 0, 528, 529, 1, 0, 0, 0, 529, 109, 1, 0, 0, 0, 530, 531, 5, 21, 0, 0, 531, 532, 3, 112, 56, 0, 532, 111, 1, 0, 0, 0, 533, 535, 3, 114, 57, 0, 534, 533, 1, 0, 0, 0, 535, 536, 1, 0, 0, 0, 536, 534, 1, 0, 0, 0, 536, 537, 1, 0, 0, 0, 537, 113, 1, 0, 0, 0, 538, 539, 5, 95, 0, 0, 539, 540, 3, 116, 58, 0, 540, 541, 5, 96, 0, 0, 541, 115, 1, 0, 0, 0, 542, 543, 6, 58, -1, 0, 543, 544, 3, 118, 59, 0, 544, 550, 1, 0, 0, 0, 545, 546, 10, 1, 0, 0, 546, 547, 5, 48, 0, 0, 547, 549, 3, 118, 59, 0, 548, 545, 1, 0, 0, 0, 549, 552, 1, 0, 0, 0, 550, 548, 1, 0, 0, 0, 550, 551, 1, 0, 0, 0, 551, 117, 1, 0, 0, 0, 552, 550, 1, 0, 0, 0, 553, 554, 3, 6, 3, 0, 554, 119, 1, 0, 0, 0, 555, 559, 5, 12, 0, 0, 556, 557, 3, 48, 24, 0, 557, 558, 5, 54, 0, 0, 558, 560, 1, 0, 0, 0, 559, 556, 1, 0, 0, 0, 559, 560, 1, 0, 0, 0, 560, 561, 1, 0, 0, 0, 561, 562, 3, 156, 78, 0, 562, 563, 5, 70, 0, 0, 563, 564, 3, 18, 9, 0, 564, 565, 3, 88, 44, 0, 565, 121, 1, 0, 0, 0, 566, 570, 5, 7, 0, 0, 567, 568, 3, 48, 24, 0, 568, 569, 5, 54, 0, 0, 569, 571, 1, 0, 0, 0, 570, 567, 1, 0, 0, 0, 570, 571, 1, 0, 0, 0, 571, 572, 1, 0, 0, 0, 572, 573, 3, 144, 72, 0, 573, 574, 3, 88, 44, 0, 574, 123, 1, 0, 0, 0, 575, 576, 5, 27, 0, 0, 576, 577, 3, 28, 14, 0, 577, 578, 5, 70, 0, 0, 578, 579, 3, 56, 28, 0, 579, 125, 1, 0, 0, 0, 580, 581, 5, 18, 0, 0, 581, 584, 3, 44, 22, 0, 582, 583, 5, 55, 0, 0, 583, 585, 3, 14, 7, 0, 584, 582, 1, 0, 0, 0, 584, 585, 1, 0, 0, 0, 585, 127, 1, 0, 0, 0, 586, 587, 5, 31, 0, 0, 587, 588, 3, 56, 28, 0, 588, 129, 1, 0, 0, 0, 589, 590, 5, 22, 0, 0, 590, 131, 1, 0, 0, 0, 591, 592, 6, 66, -1, 0, 592, 593, 5, 67, 0, 0, 593, 621, 3, 132, 66, 8, 594, 621, 3, 138, 69, 0, 595, 621, 3, 134, 67, 0, 596, 598, 3, 138, 69, 0, 597, 599, 5, 67, 0, 0, 598, 597, 1, 0, 0, 0, 598, 599, 1, 0, 0, 0, 599, 600, 1, 0, 0, 0, 600, 601, 5, 63, 0, 0, 601, 602, 5, 95, 0, 0, 602, 607, 3, 138, 69, 0, 603, 604, 5, 58, 0, 0, 604, 606, 3, 138, 69, 0, 605, 603, 1, 0, 0, 0, 606, 609, 1, 0, 0, 0, 607, 605, 1, 0, 0, 0, 607, 608, 1, 0, 0, 0, 608, 610, 1, 0, 0, 0, 609, 607, 1, 0, 0, 0, 610, 611, 5, 96, 0, 0, 611, 621, 1, 0, 0, 0, 612, 613, 3, 138, 69, 0, 613, 615, 5, 64, 0, 0, 614, 616, 5, 67, 0, 0, 615, 614, 1, 0, 0, 0, 615, 616, 1, 0, 0, 0, 616, 617, 1, 0, 0, 0, 617, 618, 5, 68, 0, 0, 618, 621, 1, 0, 0, 0, 619, 621, 3, 136, 68, 0, 620, 591, 1, 0, 0, 0, 620, 594, 1, 0, 0, 0, 620, 595, 1, 0, 0, 0, 620, 596, 1, 0, 0, 0, 620, 612, 1, 0, 0, 0, 620, 619, 1, 0, 0, 0, 621, 630, 1, 0, 0, 0, 622, 623, 10, 5, 0, 0, 623, 624, 5, 52, 0, 0, 624, 629, 3, 132, 66, 6, 625, 626, 10, 4, 0, 0, 626, 627, 5, 71, 0, 0, 627, 629, 3, 132, 66, 5, 628, 622, 1, 0, 0, 0, 628, 625, 1, 0, 0, 0, 629, 632, 1, 0, 0, 0, 630, 628, 1, 0, 0, 0, 630, 631, 1, 0, 0, 0, 631, 133, 1, 0, 0, 0, 632, 630, 1, 0, 0, 0, 633, 635, 3, 138, 69, 0, 634, 636, 5, 67, 0, 0, 635, 634, 1, 0, 0, 0, 635, 636, 1, 0, 0, 0, 636, 637, 1, 0, 0, 0, 637, 638, 5, 66, 0, 0, 638, 639, 3, 166, 83, 0, 639, 680, 1, 0, 0, 0, 640, 642, 3, 138, 69, 0, 641, 643, 5, 67, 0, 0, 642, 641, 1, 0, 0, 0, 642, 643, 1, 0, 0, 0, 643, 644, 1, 0, 0, 0, 644, 645, 5, 73, 0, 0, 645, 646, 3, 166, 83, 0, 646, 680, 1, 0, 0, 0, 647, 649, 3, 138, 69, 0, 648, 650, 5, 67, 0, 0, 649, 648, 1, 0, 0, 0, 649, 650, 1, 0, 0, 0, 650, 651, 1, 0, 0, 0, 651, 652, 5, 66, 0, 0, 652, 653, 5, 95, 0, 0, 653, 658, 3, 166, 83, 0, 654, 655, 5, 58, 0, 0, 655, 657, 3, 166, 83, 0, 656, 654, 1, 0, 0, 0, 657, 660, 1, 0, 0, 0, 658, 656, 1, 0, 0, 0, 658, 659, 1, 0, 0, 0, 659, 661, 1, 0, 0, 0, 660, 658, 1, 0, 0, 0, 661, 662, 5, 96, 0, 0, 662, 680, 1, 0, 0, 0, 663, 665, 3, 138, 69, 0, 664, 666, 5, 67, 0, 0, 665, 664, 1, 0, 0, 0, 665, 666, 1, 0, 0, 0, 666, 667, 1, 0, 0, 0, 667, 668, 5, 73, 0, 0, 668, 669, 5, 95, 0, 0, 669, 674, 3, 166, 83, 0, 670, 671, 5, 58, 0, 0, 671, 673, 3, 166, 83, 0, 672, 670, 1, 0, 0, 0, 673, 676, 1, 0, 0, 0, 674, 672, 1, 0, 0, 0, 674, 675, 1, 0, 0, 0, 675, 677, 1, 0, 0, 0, 676, 674, 1, 0, 0, 0, 677, 678, 5, 96, 0, 0, 678, 680, 1, 0, 0, 0, 679, 633, 1, 0, 0, 0, 679, 640, 1, 0, 0, 0, 679, 647, 1, 0, 0, 0, 679, 663, 1, 0, 0, 0, 680, 135, 1, 0, 0, 0, 681, 684, 3, 48, 24, 0, 682, 683, 5, 56, 0, 0, 683, 685, 3, 10, 5, 0, 684, 682, 1, 0, 0, 0, 684, 685, 1, 0, 0, 0, 685, 686, 1, 0, 0, 0, 686, 687, 5, 57, 0, 0, 687, 688, 3, 156, 78, 0, 688, 137, 1, 0, 0, 0, 689, 692, 3, 142, 71, 0, 690, 692, 3, 140, 70, 0, 691, 689, 1, 0, 0, 0, 691, 690, 1, 0, 0, 0, 692, 139, 1, 0, 0, 0, 693, 694, 3, 142, 71, 0, 694, 695, 3, 168, 84, 0, 695, 696, 3, 142, 71, 0, 696, 141, 1, 0, 0, 0, 697, 698, 6, 71, -1, 0, 698, 702, 3, 144, 72, 0, 699, 700, 7, 5, 0, 0, 700, 702, 3, 142, 71, 3, 701, 697, 1, 0, 0, 0, 701, 699, 1, 0, 0, 0, 702, 711, 1, 0, 0, 0, 703, 704, 10, 2, 0, 0, 704, 705, 7, 6, 0, 0, 705, 710, 3, 142, 71, 3, 706, 707, 10, 1, 0, 0, 707, 708, 7, 5, 0, 0, 708, 710, 3, 142, 71, 2, 709, 703, 1, 0, 0, 0, 709, 706, 1, 0, 0, 0, 710, 713, 1, 0, 0, 0, 711, 709, 1, 0, 0, 0, 711, 712, 1, 0, 0, 0, 712, 143, 1, 0, 0, 0, 713, 711, 1, 0, 0, 0, 714, 715, 6, 72, -1, 0, 715, 723, 3, 156, 78, 0, 716, 723, 3, 48, 24, 0, 717, 723, 3, 146, 73, 0, 718, 719, 5, 95, 0, 0, 719, 720, 3, 132, 66, 0, 720, 721, 5, 96, 0, 0, 721, 723, 1, 0, 0, 0, 722, 714, 1, 0, 0, 0, 722, 716, 1, 0, 0, 0, 722, 717, 1, 0, 0, 0, 722, 718, 1, 0, 0, 0, 723, 729, 1, 0, 0, 0, 724, 725, 10, 1, 0, 0, 725, 726, 5, 56, 0, 0, 726, 728, 3, 10, 5, 0, 727, 724, 1, 0, 0, 0, 728, 731, 1, 0, 0, 0, 729, 727, 1, 0, 0, 0, 729, 730, 1, 0, 0, 0, 730, 145, 1, 0, 0, 0, 731, 729, 1, 0, 0, 0, 732, 733, 3, 148, 74, 0, 733, 747, 5, 95, 0, 0, 734, 748, 5, 85, 0, 0, 735, 740, 3, 132, 66, 0, 736, 737, 5, 58, 0, 0, 737, 739, 3, 132, 66, 0, 738, 736, 1, 0, 0, 0, 739, 742, 1, 0, 0, 0, 740, 738, 1, 0, 0, 0, 740, 741, 1, 0, 0, 0, 741, 745, 1, 0, 0, 0, 742, 740, 1, 0, 0, 0, 743, 744, 5, 58, 0, 0, 744, 746, 3, 150, 75, 0, 745, 743, 1, 0, 0, 0, 745, 746, 1, 0, 0, 0, 746, 748, 1, 0, 0, 0, 747, 734, 1, 0, 0, 0, 747, 735, 1, 0, 0, 0, 747, 748, 1, 0, 0, 0, 748, 749, 1, 0, 0, 0, 749, 750, 5, 96, 0, 0, 750, 147, 1, 0, 0, 0, 751, 755, 3, 66, 33, 0, 752, 755, 5, 62, 0, 0, 753, 755, 5, 65, 0, 0, 754, 751, 1, 0, 0, 0, 754, 752, 1, 0, 0, 0, 754, 753, 1, 0, 0, 0, 755, 149, 1, 0, 0, 0, 756, 765, 5, 88, 0, 0, 757, 762, 3, 152, 76, 0, 758, 759, 5, 58, 0, 0, 759, 761, 3, 152, 76, 0, 760, 758, 1, 0, 0, 0, 761, 764, 1, 0, 0, 0, 762, 760, 1, 0, 0, 0, 762, 763, 1, 0, 0, 0, 763, 766, 1, 0, 0, 0, 764, 762, 1, 0, 0, 0, 765, 757, 1, 0, 0, 0, 765, 766, 1, 0, 0, 0, 766, 767, 1, 0, 0, 0, 767, 768, 5, 89, 0, 0, 768, 151, 1, 0, 0, 0, 769, 770, 3, 166, 83, 0, 770, 771, 5, 57, 0, 0, 771, 772, 3, 154, 77, 0, 772, 153, 1, 0, 0, 0, 773, 776, 3, 156, 78, 0, 774, 776, 3, 150, 75, 0, 775, 773, 1, 0, 0, 0, 775, 774, 1, 0, 0, 0, 776, 155, 1, 0, 0, 0, 777, 820, 5, 68, 0, 0, 778, 779, 3, 164, 82, 0, 779, 780, 5, 97, 0, 0, 780, 820, 1, 0, 0, 0, 781, 820, 3, 162, 81, 0, 782, 820, 3, 164, 82, 0, 783, 820, 3, 158, 79, 0, 784, 820, 3, 62, 31, 0, 785, 820, 3, 166, 83, 0, 786, 787, 5, 93, 0, 0, 787, 792, 3, 160, 80, 0, 788, 789, 5, 58, 0, 0, 789, 791, 3, 160, 80, 0, 790, 788, 1, 0, 0, 0, 791, 794, 1, 0, 0, 0, 792, 790, 1, 0, 0, 0, 792, 793, 1, 0, 0, 0, 793, 795, 1, 0, 0, 0, 794, 792, 1, 0, 0, 0, 795, 796, 5, 94, 0, 0, 796, 820, 1, 0, 0, 0, 797, 798, 5, 93, 0, 0, 798, 803, 3, 158, 79, 0, 799, 800, 5, 58, 0, 0, 800, 802, 3, 158, 79, 0, 801, 799, 1, 0, 0, 0, 802, 805, 1, 0, 0, 0, 803, 801, 1, 0, 0, 0, 803, 804, 1, 0, 0, 0, 804, 806, 1, 0, 0, 0, 805, 803, 1, 0, 0, 0, 806, 807, 5, 94, 0, 0, 807, 820, 1, 0, 0, 0, 808, 809, 5, 93, 0, 0, 809, 814, 3, 166, 83, 0, 810, 811, 5, 58, 0, 0, 811, 813, 3, 166, 83, 0, 812, 810, 1, 0, 0, 0, 813, 816, 1, 0, 0, 0, 814, 812, 1, 0, 0, 0, 814, 815, 1, 0, 0, 0, 815, 817, 1, 0, 0, 0, 816, 814, 1, 0, 0, 0, 817, 818, 5, 94, 0, 0, 818, 820, 1, 0, 0, 0, 819, 777, 1, 0, 0, 0, 819, 778, 1, 0, 0, 0, 819, 781, 1, 0, 0, 0, 819, 782, 1, 0, 0, 0, 819, 783, 1, 0, 0, 0, 819, 784, 1, 0, 0, 0, 819, 785, 1, 0, 0, 0, 819, 786, 1, 0, 0, 0, 819, 797, 1, 0, 0, 0, 819, 808, 1, 0, 0, 0, 820, 157, 1, 0, 0, 0, 821, 822, 7, 7, 0, 0, 822, 159, 1, 0, 0, 0, 823, 826, 3, 162, 81, 0, 824, 826, 3, 164, 82, 0, 825, 823, 1, 0, 0, 0, 825, 824, 1, 0, 0, 0, 826, 161, 1, 0, 0, 0, 827, 829, 7, 5, 0, 0, 828, 827, 1, 0, 0, 0, 828, 829, 1, 0, 0, 0, 829, 830, 1, 0, 0, 0, 830, 831, 5, 51, 0, 0, 831, 163, 1, 0, 0, 0, 832, 834, 7, 5, 0, 0, 833, 832, 1, 0, 0, 0, 833, 834, 1, 0, 0, 0, 834, 835, 1, 0, 0, 0, 835, 836, 5, 50, 0, 0, 836, 165, 1, 0, 0, 0, 837, 838, 5, 49, 0, 0, 838, 167, 1, 0, 0, 0, 839, 840, 7, 8, 0, 0, 840, 169, 1, 0, 0, 0, 841, 842, 7, 9, 0, 0, 842, 843, 5, 110, 0, 0, 843, 844, 3, 172, 86, 0, 844, 845, 3, 174, 87, 0, 845, 171, 1, 0, 0, 0, 846, 847, 4, 86, 15, 0, 847, 849, 3, 28, 14, 0, 848, 850, 5, 128, 0, 0, 849, 848, 1, 0, 0, 0, 849, 850, 1, 0, 0, 0, 850, 851, 1, 0, 0, 0, 851, 852, 5, 103, 0, 0, 852, 855, 1, 0, 0, 0, 853, 855, 3, 28, 14, 0, 854, 846, 1, 0, 0, 0, 854, 853, 1, 0, 0, 0, 855, 173, 1, 0, 0, 0, 856, 857, 5, 70, 0, 0, 857, 862, 3, 48, 24, 0, 858, 859, 5, 58, 0, 0, 859, 861, 3, 48, 24, 0, 860, 858, 1, 0, 0, 0, 861, 864, 1, 0, 0, 0, 862, 860, 1, 0, 0, 0, 862, 863, 1, 0, 0, 0, 863, 875, 1, 0, 0, 0, 864, 862, 1, 0, 0, 0, 865, 866, 5, 70, 0, 0, 866, 871, 3, 140, 70, 0, 867, 868, 5, 52, 0, 0, 868, 870, 3, 140, 70, 0, 869, 867, 1, 0, 0, 0, 870, 873, 1, 0, 0, 0, 871, 869, 1, 0, 0, 0, 871, 872, 1, 0, 0, 0, 872, 875, 1, 0, 0, 0, 873, 871, 1, 0, 0, 0, 874, 856, 1, 0, 0, 0, 874, 865, 1, 0, 0, 0, 875, 175, 1, 0, 0, 0, 84, 187, 197, 225, 240, 246, 255, 261, 274, 278, 289, 305, 313, 317, 324, 330, 335, 344, 351, 357, 366, 373, 381, 389, 393, 397, 402, 413, 418, 422, 436, 447, 453, 460, 469, 492, 500, 503, 510, 521, 528, 536, 550, 559, 570, 584, 598, 607, 615, 620, 628, 630, 635, 642, 649, 658, 665, 674, 679, 684, 691, 701, 709, 711, 722, 729, 740, 745, 747, 754, 762, 765, 775, 792, 803, 814, 819, 825, 828, 833, 849, 854, 862, 871, 874]
\ No newline at end of file
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
index 7ada3fb69a502..686b35f728cdd 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParser.java
@@ -75,12 +75,12 @@ public class EsqlBaseParser extends ParserConfig {
RULE_rerankCommand = 60, RULE_completionCommand = 61, RULE_lookupCommand = 62,
RULE_inlinestatsCommand = 63, RULE_insistCommand = 64, RULE_fuseCommand = 65,
RULE_booleanExpression = 66, RULE_regexBooleanExpression = 67, RULE_matchBooleanExpression = 68,
- RULE_valueExpression = 69, RULE_operatorExpression = 70, RULE_primaryExpression = 71,
- RULE_functionExpression = 72, RULE_functionName = 73, RULE_mapExpression = 74,
- RULE_entryExpression = 75, RULE_mapValue = 76, RULE_constant = 77, RULE_booleanValue = 78,
- RULE_numericValue = 79, RULE_decimalValue = 80, RULE_integerValue = 81,
- RULE_string = 82, RULE_comparisonOperator = 83, RULE_joinCommand = 84,
- RULE_joinTarget = 85, RULE_joinCondition = 86, RULE_joinPredicate = 87;
+ RULE_valueExpression = 69, RULE_comparisonExpression = 70, RULE_operatorExpression = 71,
+ RULE_primaryExpression = 72, RULE_functionExpression = 73, RULE_functionName = 74,
+ RULE_mapExpression = 75, RULE_entryExpression = 76, RULE_mapValue = 77,
+ RULE_constant = 78, RULE_booleanValue = 79, RULE_numericValue = 80, RULE_decimalValue = 81,
+ RULE_integerValue = 82, RULE_string = 83, RULE_comparisonOperator = 84,
+ RULE_joinCommand = 85, RULE_joinTarget = 86, RULE_joinCondition = 87;
private static String[] makeRuleNames() {
return new String[] {
"singleStatement", "query", "sourceCommand", "processingCommand", "whereCommand",
@@ -99,11 +99,11 @@ private static String[] makeRuleNames() {
"forkSubQuery", "forkSubQueryCommand", "forkSubQueryProcessingCommand",
"rerankCommand", "completionCommand", "lookupCommand", "inlinestatsCommand",
"insistCommand", "fuseCommand", "booleanExpression", "regexBooleanExpression",
- "matchBooleanExpression", "valueExpression", "operatorExpression", "primaryExpression",
- "functionExpression", "functionName", "mapExpression", "entryExpression",
- "mapValue", "constant", "booleanValue", "numericValue", "decimalValue",
- "integerValue", "string", "comparisonOperator", "joinCommand", "joinTarget",
- "joinCondition", "joinPredicate"
+ "matchBooleanExpression", "valueExpression", "comparisonExpression",
+ "operatorExpression", "primaryExpression", "functionExpression", "functionName",
+ "mapExpression", "entryExpression", "mapValue", "constant", "booleanValue",
+ "numericValue", "decimalValue", "integerValue", "string", "comparisonOperator",
+ "joinCommand", "joinTarget", "joinCondition"
};
}
public static final String[] ruleNames = makeRuleNames();
@@ -5564,16 +5564,8 @@ public T accept(ParseTreeVisitor extends T> visitor) {
}
@SuppressWarnings("CheckReturnValue")
public static class ComparisonContext extends ValueExpressionContext {
- public OperatorExpressionContext left;
- public OperatorExpressionContext right;
- public ComparisonOperatorContext comparisonOperator() {
- return getRuleContext(ComparisonOperatorContext.class,0);
- }
- public List operatorExpression() {
- return getRuleContexts(OperatorExpressionContext.class);
- }
- public OperatorExpressionContext operatorExpression(int i) {
- return getRuleContext(OperatorExpressionContext.class,i);
+ public ComparisonExpressionContext comparisonExpression() {
+ return getRuleContext(ComparisonExpressionContext.class,0);
}
@SuppressWarnings("this-escape")
public ComparisonContext(ValueExpressionContext ctx) { copyFrom(ctx); }
@@ -5596,7 +5588,7 @@ public final ValueExpressionContext valueExpression() throws RecognitionExceptio
ValueExpressionContext _localctx = new ValueExpressionContext(_ctx, getState());
enterRule(_localctx, 138, RULE_valueExpression);
try {
- setState(694);
+ setState(691);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,59,_ctx) ) {
case 1:
@@ -5612,11 +5604,7 @@ public final ValueExpressionContext valueExpression() throws RecognitionExceptio
enterOuterAlt(_localctx, 2);
{
setState(690);
- ((ComparisonContext)_localctx).left = operatorExpression(0);
- setState(691);
- comparisonOperator();
- setState(692);
- ((ComparisonContext)_localctx).right = operatorExpression(0);
+ comparisonExpression();
}
break;
}
@@ -5632,6 +5620,64 @@ public final ValueExpressionContext valueExpression() throws RecognitionExceptio
return _localctx;
}
+ @SuppressWarnings("CheckReturnValue")
+ public static class ComparisonExpressionContext extends ParserRuleContext {
+ public OperatorExpressionContext left;
+ public OperatorExpressionContext right;
+ public ComparisonOperatorContext comparisonOperator() {
+ return getRuleContext(ComparisonOperatorContext.class,0);
+ }
+ public List operatorExpression() {
+ return getRuleContexts(OperatorExpressionContext.class);
+ }
+ public OperatorExpressionContext operatorExpression(int i) {
+ return getRuleContext(OperatorExpressionContext.class,i);
+ }
+ @SuppressWarnings("this-escape")
+ public ComparisonExpressionContext(ParserRuleContext parent, int invokingState) {
+ super(parent, invokingState);
+ }
+ @Override public int getRuleIndex() { return RULE_comparisonExpression; }
+ @Override
+ public void enterRule(ParseTreeListener listener) {
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterComparisonExpression(this);
+ }
+ @Override
+ public void exitRule(ParseTreeListener listener) {
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitComparisonExpression(this);
+ }
+ @Override
+ public T accept(ParseTreeVisitor extends T> visitor) {
+ if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitComparisonExpression(this);
+ else return visitor.visitChildren(this);
+ }
+ }
+
+ public final ComparisonExpressionContext comparisonExpression() throws RecognitionException {
+ ComparisonExpressionContext _localctx = new ComparisonExpressionContext(_ctx, getState());
+ enterRule(_localctx, 140, RULE_comparisonExpression);
+ try {
+ enterOuterAlt(_localctx, 1);
+ {
+ setState(693);
+ ((ComparisonExpressionContext)_localctx).left = operatorExpression(0);
+ setState(694);
+ comparisonOperator();
+ setState(695);
+ ((ComparisonExpressionContext)_localctx).right = operatorExpression(0);
+ }
+ }
+ catch (RecognitionException re) {
+ _localctx.exception = re;
+ _errHandler.reportError(this, re);
+ _errHandler.recover(this, re);
+ }
+ finally {
+ exitRule();
+ }
+ return _localctx;
+ }
+
@SuppressWarnings("CheckReturnValue")
public static class OperatorExpressionContext extends ParserRuleContext {
@SuppressWarnings("this-escape")
@@ -5733,14 +5779,14 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
int _parentState = getState();
OperatorExpressionContext _localctx = new OperatorExpressionContext(_ctx, _parentState);
OperatorExpressionContext _prevctx = _localctx;
- int _startState = 140;
- enterRecursionRule(_localctx, 140, RULE_operatorExpression, _p);
+ int _startState = 142;
+ enterRecursionRule(_localctx, 142, RULE_operatorExpression, _p);
int _la;
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(700);
+ setState(701);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,60,_ctx) ) {
case 1:
@@ -5749,7 +5795,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_ctx = _localctx;
_prevctx = _localctx;
- setState(697);
+ setState(698);
primaryExpression(0);
}
break;
@@ -5758,7 +5804,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_localctx = new ArithmeticUnaryContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(698);
+ setState(699);
((ArithmeticUnaryContext)_localctx).operator = _input.LT(1);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
@@ -5769,13 +5815,13 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_errHandler.reportMatch(this);
consume();
}
- setState(699);
+ setState(700);
operatorExpression(3);
}
break;
}
_ctx.stop = _input.LT(-1);
- setState(710);
+ setState(711);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,62,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
@@ -5783,7 +5829,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
if ( _parseListeners!=null ) triggerExitRuleEvent();
_prevctx = _localctx;
{
- setState(708);
+ setState(709);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,61,_ctx) ) {
case 1:
@@ -5791,9 +5837,9 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_localctx = new ArithmeticBinaryContext(new OperatorExpressionContext(_parentctx, _parentState));
((ArithmeticBinaryContext)_localctx).left = _prevctx;
pushNewRecursionContext(_localctx, _startState, RULE_operatorExpression);
- setState(702);
- if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)");
setState(703);
+ if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)");
+ setState(704);
((ArithmeticBinaryContext)_localctx).operator = _input.LT(1);
_la = _input.LA(1);
if ( !(((((_la - 85)) & ~0x3f) == 0 && ((1L << (_la - 85)) & 7L) != 0)) ) {
@@ -5804,7 +5850,7 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_errHandler.reportMatch(this);
consume();
}
- setState(704);
+ setState(705);
((ArithmeticBinaryContext)_localctx).right = operatorExpression(3);
}
break;
@@ -5813,9 +5859,9 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_localctx = new ArithmeticBinaryContext(new OperatorExpressionContext(_parentctx, _parentState));
((ArithmeticBinaryContext)_localctx).left = _prevctx;
pushNewRecursionContext(_localctx, _startState, RULE_operatorExpression);
- setState(705);
- if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
setState(706);
+ if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
+ setState(707);
((ArithmeticBinaryContext)_localctx).operator = _input.LT(1);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
@@ -5826,14 +5872,14 @@ private OperatorExpressionContext operatorExpression(int _p) throws RecognitionE
_errHandler.reportMatch(this);
consume();
}
- setState(707);
+ setState(708);
((ArithmeticBinaryContext)_localctx).right = operatorExpression(2);
}
break;
}
}
}
- setState(712);
+ setState(713);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,62,_ctx);
}
@@ -5985,13 +6031,13 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc
int _parentState = getState();
PrimaryExpressionContext _localctx = new PrimaryExpressionContext(_ctx, _parentState);
PrimaryExpressionContext _prevctx = _localctx;
- int _startState = 142;
- enterRecursionRule(_localctx, 142, RULE_primaryExpression, _p);
+ int _startState = 144;
+ enterRecursionRule(_localctx, 144, RULE_primaryExpression, _p);
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(721);
+ setState(722);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,63,_ctx) ) {
case 1:
@@ -6000,7 +6046,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc
_ctx = _localctx;
_prevctx = _localctx;
- setState(714);
+ setState(715);
constant();
}
break;
@@ -6009,7 +6055,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc
_localctx = new DereferenceContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(715);
+ setState(716);
qualifiedName();
}
break;
@@ -6018,7 +6064,7 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc
_localctx = new FunctionContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(716);
+ setState(717);
functionExpression();
}
break;
@@ -6027,17 +6073,17 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc
_localctx = new ParenthesizedExpressionContext(_localctx);
_ctx = _localctx;
_prevctx = _localctx;
- setState(717);
- match(LP);
setState(718);
- booleanExpression(0);
+ match(LP);
setState(719);
+ booleanExpression(0);
+ setState(720);
match(RP);
}
break;
}
_ctx.stop = _input.LT(-1);
- setState(728);
+ setState(729);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,64,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
@@ -6048,16 +6094,16 @@ private PrimaryExpressionContext primaryExpression(int _p) throws RecognitionExc
{
_localctx = new InlineCastContext(new PrimaryExpressionContext(_parentctx, _parentState));
pushNewRecursionContext(_localctx, _startState, RULE_primaryExpression);
- setState(723);
- if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
setState(724);
- match(CAST_OP);
+ if (!(precpred(_ctx, 1))) throw new FailedPredicateException(this, "precpred(_ctx, 1)");
setState(725);
+ match(CAST_OP);
+ setState(726);
dataType();
}
}
}
- setState(730);
+ setState(731);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,64,_ctx);
}
@@ -6117,56 +6163,56 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final FunctionExpressionContext functionExpression() throws RecognitionException {
FunctionExpressionContext _localctx = new FunctionExpressionContext(_ctx, getState());
- enterRule(_localctx, 144, RULE_functionExpression);
+ enterRule(_localctx, 146, RULE_functionExpression);
int _la;
try {
int _alt;
enterOuterAlt(_localctx, 1);
{
- setState(731);
- functionName();
setState(732);
+ functionName();
+ setState(733);
match(LP);
- setState(746);
+ setState(747);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,67,_ctx) ) {
case 1:
{
- setState(733);
+ setState(734);
match(ASTERISK);
}
break;
case 2:
{
{
- setState(734);
+ setState(735);
booleanExpression(0);
- setState(739);
+ setState(740);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,65,_ctx);
while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
if ( _alt==1 ) {
{
{
- setState(735);
- match(COMMA);
setState(736);
+ match(COMMA);
+ setState(737);
booleanExpression(0);
}
}
}
- setState(741);
+ setState(742);
_errHandler.sync(this);
_alt = getInterpreter().adaptivePredict(_input,65,_ctx);
}
- setState(744);
+ setState(745);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==COMMA) {
{
- setState(742);
- match(COMMA);
setState(743);
+ match(COMMA);
+ setState(744);
mapExpression();
}
}
@@ -6175,7 +6221,7 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx
}
break;
}
- setState(748);
+ setState(749);
match(RP);
}
}
@@ -6219,9 +6265,9 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final FunctionNameContext functionName() throws RecognitionException {
FunctionNameContext _localctx = new FunctionNameContext(_ctx, getState());
- enterRule(_localctx, 146, RULE_functionName);
+ enterRule(_localctx, 148, RULE_functionName);
try {
- setState(753);
+ setState(754);
_errHandler.sync(this);
switch (_input.LA(1)) {
case PARAM:
@@ -6232,21 +6278,21 @@ public final FunctionNameContext functionName() throws RecognitionException {
case QUOTED_IDENTIFIER:
enterOuterAlt(_localctx, 1);
{
- setState(750);
+ setState(751);
identifierOrParameter();
}
break;
case FIRST:
enterOuterAlt(_localctx, 2);
{
- setState(751);
+ setState(752);
match(FIRST);
}
break;
case LAST:
enterOuterAlt(_localctx, 3);
{
- setState(752);
+ setState(753);
match(LAST);
}
break;
@@ -6301,40 +6347,40 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final MapExpressionContext mapExpression() throws RecognitionException {
MapExpressionContext _localctx = new MapExpressionContext(_ctx, getState());
- enterRule(_localctx, 148, RULE_mapExpression);
+ enterRule(_localctx, 150, RULE_mapExpression);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(755);
+ setState(756);
match(LEFT_BRACES);
- setState(764);
+ setState(765);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==QUOTED_STRING) {
{
- setState(756);
+ setState(757);
entryExpression();
- setState(761);
+ setState(762);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(757);
- match(COMMA);
setState(758);
+ match(COMMA);
+ setState(759);
entryExpression();
}
}
- setState(763);
+ setState(764);
_errHandler.sync(this);
_la = _input.LA(1);
}
}
}
- setState(766);
+ setState(767);
match(RIGHT_BRACES);
}
}
@@ -6382,15 +6428,15 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final EntryExpressionContext entryExpression() throws RecognitionException {
EntryExpressionContext _localctx = new EntryExpressionContext(_ctx, getState());
- enterRule(_localctx, 150, RULE_entryExpression);
+ enterRule(_localctx, 152, RULE_entryExpression);
try {
enterOuterAlt(_localctx, 1);
{
- setState(768);
- ((EntryExpressionContext)_localctx).key = string();
setState(769);
- match(COLON);
+ ((EntryExpressionContext)_localctx).key = string();
setState(770);
+ match(COLON);
+ setState(771);
((EntryExpressionContext)_localctx).value = mapValue();
}
}
@@ -6435,9 +6481,9 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final MapValueContext mapValue() throws RecognitionException {
MapValueContext _localctx = new MapValueContext(_ctx, getState());
- enterRule(_localctx, 152, RULE_mapValue);
+ enterRule(_localctx, 154, RULE_mapValue);
try {
- setState(774);
+ setState(775);
_errHandler.sync(this);
switch (_input.LA(1)) {
case QUOTED_STRING:
@@ -6453,14 +6499,14 @@ public final MapValueContext mapValue() throws RecognitionException {
case OPENING_BRACKET:
enterOuterAlt(_localctx, 1);
{
- setState(772);
+ setState(773);
constant();
}
break;
case LEFT_BRACES:
enterOuterAlt(_localctx, 2);
{
- setState(773);
+ setState(774);
mapExpression();
}
break;
@@ -6732,17 +6778,17 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final ConstantContext constant() throws RecognitionException {
ConstantContext _localctx = new ConstantContext(_ctx, getState());
- enterRule(_localctx, 154, RULE_constant);
+ enterRule(_localctx, 156, RULE_constant);
int _la;
try {
- setState(818);
+ setState(819);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,75,_ctx) ) {
case 1:
_localctx = new NullLiteralContext(_localctx);
enterOuterAlt(_localctx, 1);
{
- setState(776);
+ setState(777);
match(NULL);
}
break;
@@ -6750,9 +6796,9 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new QualifiedIntegerLiteralContext(_localctx);
enterOuterAlt(_localctx, 2);
{
- setState(777);
- integerValue();
setState(778);
+ integerValue();
+ setState(779);
match(UNQUOTED_IDENTIFIER);
}
break;
@@ -6760,7 +6806,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new DecimalLiteralContext(_localctx);
enterOuterAlt(_localctx, 3);
{
- setState(780);
+ setState(781);
decimalValue();
}
break;
@@ -6768,7 +6814,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new IntegerLiteralContext(_localctx);
enterOuterAlt(_localctx, 4);
{
- setState(781);
+ setState(782);
integerValue();
}
break;
@@ -6776,7 +6822,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new BooleanLiteralContext(_localctx);
enterOuterAlt(_localctx, 5);
{
- setState(782);
+ setState(783);
booleanValue();
}
break;
@@ -6784,7 +6830,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new InputParameterContext(_localctx);
enterOuterAlt(_localctx, 6);
{
- setState(783);
+ setState(784);
parameter();
}
break;
@@ -6792,7 +6838,7 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new StringLiteralContext(_localctx);
enterOuterAlt(_localctx, 7);
{
- setState(784);
+ setState(785);
string();
}
break;
@@ -6800,27 +6846,27 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new NumericArrayLiteralContext(_localctx);
enterOuterAlt(_localctx, 8);
{
- setState(785);
- match(OPENING_BRACKET);
setState(786);
+ match(OPENING_BRACKET);
+ setState(787);
numericValue();
- setState(791);
+ setState(792);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(787);
- match(COMMA);
setState(788);
+ match(COMMA);
+ setState(789);
numericValue();
}
}
- setState(793);
+ setState(794);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(794);
+ setState(795);
match(CLOSING_BRACKET);
}
break;
@@ -6828,27 +6874,27 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new BooleanArrayLiteralContext(_localctx);
enterOuterAlt(_localctx, 9);
{
- setState(796);
- match(OPENING_BRACKET);
setState(797);
+ match(OPENING_BRACKET);
+ setState(798);
booleanValue();
- setState(802);
+ setState(803);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(798);
- match(COMMA);
setState(799);
+ match(COMMA);
+ setState(800);
booleanValue();
}
}
- setState(804);
+ setState(805);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(805);
+ setState(806);
match(CLOSING_BRACKET);
}
break;
@@ -6856,27 +6902,27 @@ public final ConstantContext constant() throws RecognitionException {
_localctx = new StringArrayLiteralContext(_localctx);
enterOuterAlt(_localctx, 10);
{
- setState(807);
- match(OPENING_BRACKET);
setState(808);
+ match(OPENING_BRACKET);
+ setState(809);
string();
- setState(813);
+ setState(814);
_errHandler.sync(this);
_la = _input.LA(1);
while (_la==COMMA) {
{
{
- setState(809);
- match(COMMA);
setState(810);
+ match(COMMA);
+ setState(811);
string();
}
}
- setState(815);
+ setState(816);
_errHandler.sync(this);
_la = _input.LA(1);
}
- setState(816);
+ setState(817);
match(CLOSING_BRACKET);
}
break;
@@ -6919,12 +6965,12 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final BooleanValueContext booleanValue() throws RecognitionException {
BooleanValueContext _localctx = new BooleanValueContext(_ctx, getState());
- enterRule(_localctx, 156, RULE_booleanValue);
+ enterRule(_localctx, 158, RULE_booleanValue);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(820);
+ setState(821);
_la = _input.LA(1);
if ( !(_la==FALSE || _la==TRUE) ) {
_errHandler.recoverInline(this);
@@ -6977,22 +7023,22 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final NumericValueContext numericValue() throws RecognitionException {
NumericValueContext _localctx = new NumericValueContext(_ctx, getState());
- enterRule(_localctx, 158, RULE_numericValue);
+ enterRule(_localctx, 160, RULE_numericValue);
try {
- setState(824);
+ setState(825);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,76,_ctx) ) {
case 1:
enterOuterAlt(_localctx, 1);
{
- setState(822);
+ setState(823);
decimalValue();
}
break;
case 2:
enterOuterAlt(_localctx, 2);
{
- setState(823);
+ setState(824);
integerValue();
}
break;
@@ -7036,17 +7082,17 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final DecimalValueContext decimalValue() throws RecognitionException {
DecimalValueContext _localctx = new DecimalValueContext(_ctx, getState());
- enterRule(_localctx, 160, RULE_decimalValue);
+ enterRule(_localctx, 162, RULE_decimalValue);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(827);
+ setState(828);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==PLUS || _la==MINUS) {
{
- setState(826);
+ setState(827);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
_errHandler.recoverInline(this);
@@ -7059,7 +7105,7 @@ public final DecimalValueContext decimalValue() throws RecognitionException {
}
}
- setState(829);
+ setState(830);
match(DECIMAL_LITERAL);
}
}
@@ -7101,17 +7147,17 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final IntegerValueContext integerValue() throws RecognitionException {
IntegerValueContext _localctx = new IntegerValueContext(_ctx, getState());
- enterRule(_localctx, 162, RULE_integerValue);
+ enterRule(_localctx, 164, RULE_integerValue);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(832);
+ setState(833);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==PLUS || _la==MINUS) {
{
- setState(831);
+ setState(832);
_la = _input.LA(1);
if ( !(_la==PLUS || _la==MINUS) ) {
_errHandler.recoverInline(this);
@@ -7124,7 +7170,7 @@ public final IntegerValueContext integerValue() throws RecognitionException {
}
}
- setState(834);
+ setState(835);
match(INTEGER_LITERAL);
}
}
@@ -7164,11 +7210,11 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final StringContext string() throws RecognitionException {
StringContext _localctx = new StringContext(_ctx, getState());
- enterRule(_localctx, 164, RULE_string);
+ enterRule(_localctx, 166, RULE_string);
try {
enterOuterAlt(_localctx, 1);
{
- setState(836);
+ setState(837);
match(QUOTED_STRING);
}
}
@@ -7213,12 +7259,12 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final ComparisonOperatorContext comparisonOperator() throws RecognitionException {
ComparisonOperatorContext _localctx = new ComparisonOperatorContext(_ctx, getState());
- enterRule(_localctx, 166, RULE_comparisonOperator);
+ enterRule(_localctx, 168, RULE_comparisonOperator);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(838);
+ setState(839);
_la = _input.LA(1);
if ( !(((((_la - 76)) & ~0x3f) == 0 && ((1L << (_la - 76)) & 125L) != 0)) ) {
_errHandler.recoverInline(this);
@@ -7276,12 +7322,12 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final JoinCommandContext joinCommand() throws RecognitionException {
JoinCommandContext _localctx = new JoinCommandContext(_ctx, getState());
- enterRule(_localctx, 168, RULE_joinCommand);
+ enterRule(_localctx, 170, RULE_joinCommand);
int _la;
try {
enterOuterAlt(_localctx, 1);
{
- setState(840);
+ setState(841);
((JoinCommandContext)_localctx).type = _input.LT(1);
_la = _input.LA(1);
if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & 109051904L) != 0)) ) {
@@ -7292,11 +7338,11 @@ public final JoinCommandContext joinCommand() throws RecognitionException {
_errHandler.reportMatch(this);
consume();
}
- setState(841);
- match(JOIN);
setState(842);
- joinTarget();
+ match(JOIN);
setState(843);
+ joinTarget();
+ setState(844);
joinCondition();
}
}
@@ -7342,37 +7388,37 @@ public T accept(ParseTreeVisitor extends T> visitor) {
public final JoinTargetContext joinTarget() throws RecognitionException {
JoinTargetContext _localctx = new JoinTargetContext(_ctx, getState());
- enterRule(_localctx, 170, RULE_joinTarget);
+ enterRule(_localctx, 172, RULE_joinTarget);
int _la;
try {
- setState(853);
+ setState(854);
_errHandler.sync(this);
switch ( getInterpreter().adaptivePredict(_input,80,_ctx) ) {
case 1:
enterOuterAlt(_localctx, 1);
{
- setState(845);
- if (!(this.isDevVersion())) throw new FailedPredicateException(this, "this.isDevVersion()");
setState(846);
+ if (!(this.isDevVersion())) throw new FailedPredicateException(this, "this.isDevVersion()");
+ setState(847);
((JoinTargetContext)_localctx).index = indexPattern();
- setState(848);
+ setState(849);
_errHandler.sync(this);
_la = _input.LA(1);
if (_la==AS) {
{
- setState(847);
+ setState(848);
match(AS);
}
}
- setState(850);
+ setState(851);
((JoinTargetContext)_localctx).qualifier = match(UNQUOTED_SOURCE);
}
break;
case 2:
enterOuterAlt(_localctx, 2);
{
- setState(852);
+ setState(853);
((JoinTargetContext)_localctx).index = indexPattern();
}
break;
@@ -7391,112 +7437,141 @@ public final JoinTargetContext joinTarget() throws RecognitionException {
@SuppressWarnings("CheckReturnValue")
public static class JoinConditionContext extends ParserRuleContext {
+ @SuppressWarnings("this-escape")
+ public JoinConditionContext(ParserRuleContext parent, int invokingState) {
+ super(parent, invokingState);
+ }
+ @Override public int getRuleIndex() { return RULE_joinCondition; }
+
+ @SuppressWarnings("this-escape")
+ public JoinConditionContext() { }
+ public void copyFrom(JoinConditionContext ctx) {
+ super.copyFrom(ctx);
+ }
+ }
+ @SuppressWarnings("CheckReturnValue")
+ public static class FieldBasedLookupJoinContext extends JoinConditionContext {
public TerminalNode ON() { return getToken(EsqlBaseParser.ON, 0); }
- public List joinPredicate() {
- return getRuleContexts(JoinPredicateContext.class);
+ public List qualifiedName() {
+ return getRuleContexts(QualifiedNameContext.class);
}
- public JoinPredicateContext joinPredicate(int i) {
- return getRuleContext(JoinPredicateContext.class,i);
+ public QualifiedNameContext qualifiedName(int i) {
+ return getRuleContext(QualifiedNameContext.class,i);
}
public List COMMA() { return getTokens(EsqlBaseParser.COMMA); }
public TerminalNode COMMA(int i) {
return getToken(EsqlBaseParser.COMMA, i);
}
@SuppressWarnings("this-escape")
- public JoinConditionContext(ParserRuleContext parent, int invokingState) {
- super(parent, invokingState);
- }
- @Override public int getRuleIndex() { return RULE_joinCondition; }
+ public FieldBasedLookupJoinContext(JoinConditionContext ctx) { copyFrom(ctx); }
@Override
public void enterRule(ParseTreeListener listener) {
- if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterJoinCondition(this);
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterFieldBasedLookupJoin(this);
}
@Override
public void exitRule(ParseTreeListener listener) {
- if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitJoinCondition(this);
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitFieldBasedLookupJoin(this);
}
@Override
public T accept(ParseTreeVisitor extends T> visitor) {
- if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitJoinCondition(this);
+ if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitFieldBasedLookupJoin(this);
else return visitor.visitChildren(this);
}
}
-
- public final JoinConditionContext joinCondition() throws RecognitionException {
- JoinConditionContext _localctx = new JoinConditionContext(_ctx, getState());
- enterRule(_localctx, 172, RULE_joinCondition);
- try {
- int _alt;
- enterOuterAlt(_localctx, 1);
- {
- setState(855);
- match(ON);
- setState(856);
- joinPredicate();
- setState(861);
- _errHandler.sync(this);
- _alt = getInterpreter().adaptivePredict(_input,81,_ctx);
- while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
- if ( _alt==1 ) {
- {
- {
- setState(857);
- match(COMMA);
- setState(858);
- joinPredicate();
- }
- }
- }
- setState(863);
- _errHandler.sync(this);
- _alt = getInterpreter().adaptivePredict(_input,81,_ctx);
- }
- }
+ @SuppressWarnings("CheckReturnValue")
+ public static class ExpressionBasedLookupJoinContext extends JoinConditionContext {
+ public TerminalNode ON() { return getToken(EsqlBaseParser.ON, 0); }
+ public List comparisonExpression() {
+ return getRuleContexts(ComparisonExpressionContext.class);
}
- catch (RecognitionException re) {
- _localctx.exception = re;
- _errHandler.reportError(this, re);
- _errHandler.recover(this, re);
+ public ComparisonExpressionContext comparisonExpression(int i) {
+ return getRuleContext(ComparisonExpressionContext.class,i);
}
- finally {
- exitRule();
- }
- return _localctx;
- }
-
- @SuppressWarnings("CheckReturnValue")
- public static class JoinPredicateContext extends ParserRuleContext {
- public ValueExpressionContext valueExpression() {
- return getRuleContext(ValueExpressionContext.class,0);
+ public List AND() { return getTokens(EsqlBaseParser.AND); }
+ public TerminalNode AND(int i) {
+ return getToken(EsqlBaseParser.AND, i);
}
@SuppressWarnings("this-escape")
- public JoinPredicateContext(ParserRuleContext parent, int invokingState) {
- super(parent, invokingState);
- }
- @Override public int getRuleIndex() { return RULE_joinPredicate; }
+ public ExpressionBasedLookupJoinContext(JoinConditionContext ctx) { copyFrom(ctx); }
@Override
public void enterRule(ParseTreeListener listener) {
- if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterJoinPredicate(this);
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).enterExpressionBasedLookupJoin(this);
}
@Override
public void exitRule(ParseTreeListener listener) {
- if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitJoinPredicate(this);
+ if ( listener instanceof EsqlBaseParserListener ) ((EsqlBaseParserListener)listener).exitExpressionBasedLookupJoin(this);
}
@Override
public T accept(ParseTreeVisitor extends T> visitor) {
- if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitJoinPredicate(this);
+ if ( visitor instanceof EsqlBaseParserVisitor ) return ((EsqlBaseParserVisitor extends T>)visitor).visitExpressionBasedLookupJoin(this);
else return visitor.visitChildren(this);
}
}
- public final JoinPredicateContext joinPredicate() throws RecognitionException {
- JoinPredicateContext _localctx = new JoinPredicateContext(_ctx, getState());
- enterRule(_localctx, 174, RULE_joinPredicate);
+ public final JoinConditionContext joinCondition() throws RecognitionException {
+ JoinConditionContext _localctx = new JoinConditionContext(_ctx, getState());
+ enterRule(_localctx, 174, RULE_joinCondition);
try {
- enterOuterAlt(_localctx, 1);
- {
- setState(864);
- valueExpression();
+ int _alt;
+ setState(874);
+ _errHandler.sync(this);
+ switch ( getInterpreter().adaptivePredict(_input,83,_ctx) ) {
+ case 1:
+ _localctx = new FieldBasedLookupJoinContext(_localctx);
+ enterOuterAlt(_localctx, 1);
+ {
+ setState(856);
+ match(ON);
+ setState(857);
+ qualifiedName();
+ setState(862);
+ _errHandler.sync(this);
+ _alt = getInterpreter().adaptivePredict(_input,81,_ctx);
+ while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
+ if ( _alt==1 ) {
+ {
+ {
+ setState(858);
+ match(COMMA);
+ setState(859);
+ qualifiedName();
+ }
+ }
+ }
+ setState(864);
+ _errHandler.sync(this);
+ _alt = getInterpreter().adaptivePredict(_input,81,_ctx);
+ }
+ }
+ break;
+ case 2:
+ _localctx = new ExpressionBasedLookupJoinContext(_localctx);
+ enterOuterAlt(_localctx, 2);
+ {
+ setState(865);
+ match(ON);
+ setState(866);
+ comparisonExpression();
+ setState(871);
+ _errHandler.sync(this);
+ _alt = getInterpreter().adaptivePredict(_input,82,_ctx);
+ while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) {
+ if ( _alt==1 ) {
+ {
+ {
+ setState(867);
+ match(AND);
+ setState(868);
+ comparisonExpression();
+ }
+ }
+ }
+ setState(873);
+ _errHandler.sync(this);
+ _alt = getInterpreter().adaptivePredict(_input,82,_ctx);
+ }
+ }
+ break;
}
}
catch (RecognitionException re) {
@@ -7526,11 +7601,11 @@ public boolean sempred(RuleContext _localctx, int ruleIndex, int predIndex) {
return forkSubQueryCommand_sempred((ForkSubQueryCommandContext)_localctx, predIndex);
case 66:
return booleanExpression_sempred((BooleanExpressionContext)_localctx, predIndex);
- case 70:
- return operatorExpression_sempred((OperatorExpressionContext)_localctx, predIndex);
case 71:
+ return operatorExpression_sempred((OperatorExpressionContext)_localctx, predIndex);
+ case 72:
return primaryExpression_sempred((PrimaryExpressionContext)_localctx, predIndex);
- case 85:
+ case 86:
return joinTarget_sempred((JoinTargetContext)_localctx, predIndex);
}
return true;
@@ -7619,7 +7694,7 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
}
public static final String _serializedATN =
- "\u0004\u0001\u0087\u0363\u0002\u0000\u0007\u0000\u0002\u0001\u0007\u0001"+
+ "\u0004\u0001\u0087\u036d\u0002\u0000\u0007\u0000\u0002\u0001\u0007\u0001"+
"\u0002\u0002\u0007\u0002\u0002\u0003\u0007\u0003\u0002\u0004\u0007\u0004"+
"\u0002\u0005\u0007\u0005\u0002\u0006\u0007\u0006\u0002\u0007\u0007\u0007"+
"\u0002\b\u0007\b\u0002\t\u0007\t\u0002\n\u0007\n\u0002\u000b\u0007\u000b"+
@@ -7704,31 +7779,32 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"\bC\nC\fC\u0294\tC\u0001C\u0001C\u0001C\u0001C\u0003C\u029a\bC\u0001C"+
"\u0001C\u0001C\u0001C\u0001C\u0005C\u02a1\bC\nC\fC\u02a4\tC\u0001C\u0001"+
"C\u0003C\u02a8\bC\u0001D\u0001D\u0001D\u0003D\u02ad\bD\u0001D\u0001D\u0001"+
- "D\u0001E\u0001E\u0001E\u0001E\u0001E\u0003E\u02b7\bE\u0001F\u0001F\u0001"+
- "F\u0001F\u0003F\u02bd\bF\u0001F\u0001F\u0001F\u0001F\u0001F\u0001F\u0005"+
- "F\u02c5\bF\nF\fF\u02c8\tF\u0001G\u0001G\u0001G\u0001G\u0001G\u0001G\u0001"+
- "G\u0001G\u0003G\u02d2\bG\u0001G\u0001G\u0001G\u0005G\u02d7\bG\nG\fG\u02da"+
- "\tG\u0001H\u0001H\u0001H\u0001H\u0001H\u0001H\u0005H\u02e2\bH\nH\fH\u02e5"+
- "\tH\u0001H\u0001H\u0003H\u02e9\bH\u0003H\u02eb\bH\u0001H\u0001H\u0001"+
- "I\u0001I\u0001I\u0003I\u02f2\bI\u0001J\u0001J\u0001J\u0001J\u0005J\u02f8"+
- "\bJ\nJ\fJ\u02fb\tJ\u0003J\u02fd\bJ\u0001J\u0001J\u0001K\u0001K\u0001K"+
- "\u0001K\u0001L\u0001L\u0003L\u0307\bL\u0001M\u0001M\u0001M\u0001M\u0001"+
- "M\u0001M\u0001M\u0001M\u0001M\u0001M\u0001M\u0001M\u0001M\u0005M\u0316"+
- "\bM\nM\fM\u0319\tM\u0001M\u0001M\u0001M\u0001M\u0001M\u0001M\u0005M\u0321"+
- "\bM\nM\fM\u0324\tM\u0001M\u0001M\u0001M\u0001M\u0001M\u0001M\u0005M\u032c"+
- "\bM\nM\fM\u032f\tM\u0001M\u0001M\u0003M\u0333\bM\u0001N\u0001N\u0001O"+
- "\u0001O\u0003O\u0339\bO\u0001P\u0003P\u033c\bP\u0001P\u0001P\u0001Q\u0003"+
- "Q\u0341\bQ\u0001Q\u0001Q\u0001R\u0001R\u0001S\u0001S\u0001T\u0001T\u0001"+
- "T\u0001T\u0001T\u0001U\u0001U\u0001U\u0003U\u0351\bU\u0001U\u0001U\u0001"+
- "U\u0003U\u0356\bU\u0001V\u0001V\u0001V\u0001V\u0005V\u035c\bV\nV\fV\u035f"+
- "\tV\u0001W\u0001W\u0001W\u0000\u0005\u0002t\u0084\u008c\u008eX\u0000\u0002"+
- "\u0004\u0006\b\n\f\u000e\u0010\u0012\u0014\u0016\u0018\u001a\u001c\u001e"+
- " \"$&(*,.02468:<>@BDFHJLNPRTVXZ\\^`bdfhjlnprtvxz|~\u0080\u0082\u0084\u0086"+
- "\u0088\u008a\u008c\u008e\u0090\u0092\u0094\u0096\u0098\u009a\u009c\u009e"+
- "\u00a0\u00a2\u00a4\u00a6\u00a8\u00aa\u00ac\u00ae\u0000\n\u0002\u00001"+
- "1gg\u0001\u0000ab\u0002\u000055;;\u0002\u0000>>AA\u0002\u0000&&11\u0001"+
+ "D\u0001E\u0001E\u0003E\u02b4\bE\u0001F\u0001F\u0001F\u0001F\u0001G\u0001"+
+ "G\u0001G\u0001G\u0003G\u02be\bG\u0001G\u0001G\u0001G\u0001G\u0001G\u0001"+
+ "G\u0005G\u02c6\bG\nG\fG\u02c9\tG\u0001H\u0001H\u0001H\u0001H\u0001H\u0001"+
+ "H\u0001H\u0001H\u0003H\u02d3\bH\u0001H\u0001H\u0001H\u0005H\u02d8\bH\n"+
+ "H\fH\u02db\tH\u0001I\u0001I\u0001I\u0001I\u0001I\u0001I\u0005I\u02e3\b"+
+ "I\nI\fI\u02e6\tI\u0001I\u0001I\u0003I\u02ea\bI\u0003I\u02ec\bI\u0001I"+
+ "\u0001I\u0001J\u0001J\u0001J\u0003J\u02f3\bJ\u0001K\u0001K\u0001K\u0001"+
+ "K\u0005K\u02f9\bK\nK\fK\u02fc\tK\u0003K\u02fe\bK\u0001K\u0001K\u0001L"+
+ "\u0001L\u0001L\u0001L\u0001M\u0001M\u0003M\u0308\bM\u0001N\u0001N\u0001"+
+ "N\u0001N\u0001N\u0001N\u0001N\u0001N\u0001N\u0001N\u0001N\u0001N\u0001"+
+ "N\u0005N\u0317\bN\nN\fN\u031a\tN\u0001N\u0001N\u0001N\u0001N\u0001N\u0001"+
+ "N\u0005N\u0322\bN\nN\fN\u0325\tN\u0001N\u0001N\u0001N\u0001N\u0001N\u0001"+
+ "N\u0005N\u032d\bN\nN\fN\u0330\tN\u0001N\u0001N\u0003N\u0334\bN\u0001O"+
+ "\u0001O\u0001P\u0001P\u0003P\u033a\bP\u0001Q\u0003Q\u033d\bQ\u0001Q\u0001"+
+ "Q\u0001R\u0003R\u0342\bR\u0001R\u0001R\u0001S\u0001S\u0001T\u0001T\u0001"+
+ "U\u0001U\u0001U\u0001U\u0001U\u0001V\u0001V\u0001V\u0003V\u0352\bV\u0001"+
+ "V\u0001V\u0001V\u0003V\u0357\bV\u0001W\u0001W\u0001W\u0001W\u0005W\u035d"+
+ "\bW\nW\fW\u0360\tW\u0001W\u0001W\u0001W\u0001W\u0005W\u0366\bW\nW\fW\u0369"+
+ "\tW\u0003W\u036b\bW\u0001W\u0000\u0005\u0002t\u0084\u008e\u0090X\u0000"+
+ "\u0002\u0004\u0006\b\n\f\u000e\u0010\u0012\u0014\u0016\u0018\u001a\u001c"+
+ "\u001e \"$&(*,.02468:<>@BDFHJLNPRTVXZ\\^`bdfhjlnprtvxz|~\u0080\u0082\u0084"+
+ "\u0086\u0088\u008a\u008c\u008e\u0090\u0092\u0094\u0096\u0098\u009a\u009c"+
+ "\u009e\u00a0\u00a2\u00a4\u00a6\u00a8\u00aa\u00ac\u00ae\u0000\n\u0002\u0000"+
+ "11gg\u0001\u0000ab\u0002\u000055;;\u0002\u0000>>AA\u0002\u0000&&11\u0001"+
"\u0000ST\u0001\u0000UW\u0002\u0000==JJ\u0002\u0000LLNR\u0002\u0000\u0017"+
- "\u0017\u0019\u001a\u0388\u0000\u00b0\u0001\u0000\u0000\u0000\u0002\u00b3"+
+ "\u0017\u0019\u001a\u0394\u0000\u00b0\u0001\u0000\u0000\u0000\u0002\u00b3"+
"\u0001\u0000\u0000\u0000\u0004\u00c5\u0001\u0000\u0000\u0000\u0006\u00e1"+
"\u0001\u0000\u0000\u0000\b\u00e3\u0001\u0000\u0000\u0000\n\u00e6\u0001"+
"\u0000\u0000\u0000\f\u00e8\u0001\u0000\u0000\u0000\u000e\u00eb\u0001\u0000"+
@@ -7759,16 +7835,16 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"|\u023f\u0001\u0000\u0000\u0000~\u0244\u0001\u0000\u0000\u0000\u0080\u024a"+
"\u0001\u0000\u0000\u0000\u0082\u024d\u0001\u0000\u0000\u0000\u0084\u026c"+
"\u0001\u0000\u0000\u0000\u0086\u02a7\u0001\u0000\u0000\u0000\u0088\u02a9"+
- "\u0001\u0000\u0000\u0000\u008a\u02b6\u0001\u0000\u0000\u0000\u008c\u02bc"+
- "\u0001\u0000\u0000\u0000\u008e\u02d1\u0001\u0000\u0000\u0000\u0090\u02db"+
- "\u0001\u0000\u0000\u0000\u0092\u02f1\u0001\u0000\u0000\u0000\u0094\u02f3"+
- "\u0001\u0000\u0000\u0000\u0096\u0300\u0001\u0000\u0000\u0000\u0098\u0306"+
- "\u0001\u0000\u0000\u0000\u009a\u0332\u0001\u0000\u0000\u0000\u009c\u0334"+
- "\u0001\u0000\u0000\u0000\u009e\u0338\u0001\u0000\u0000\u0000\u00a0\u033b"+
- "\u0001\u0000\u0000\u0000\u00a2\u0340\u0001\u0000\u0000\u0000\u00a4\u0344"+
- "\u0001\u0000\u0000\u0000\u00a6\u0346\u0001\u0000\u0000\u0000\u00a8\u0348"+
- "\u0001\u0000\u0000\u0000\u00aa\u0355\u0001\u0000\u0000\u0000\u00ac\u0357"+
- "\u0001\u0000\u0000\u0000\u00ae\u0360\u0001\u0000\u0000\u0000\u00b0\u00b1"+
+ "\u0001\u0000\u0000\u0000\u008a\u02b3\u0001\u0000\u0000\u0000\u008c\u02b5"+
+ "\u0001\u0000\u0000\u0000\u008e\u02bd\u0001\u0000\u0000\u0000\u0090\u02d2"+
+ "\u0001\u0000\u0000\u0000\u0092\u02dc\u0001\u0000\u0000\u0000\u0094\u02f2"+
+ "\u0001\u0000\u0000\u0000\u0096\u02f4\u0001\u0000\u0000\u0000\u0098\u0301"+
+ "\u0001\u0000\u0000\u0000\u009a\u0307\u0001\u0000\u0000\u0000\u009c\u0333"+
+ "\u0001\u0000\u0000\u0000\u009e\u0335\u0001\u0000\u0000\u0000\u00a0\u0339"+
+ "\u0001\u0000\u0000\u0000\u00a2\u033c\u0001\u0000\u0000\u0000\u00a4\u0341"+
+ "\u0001\u0000\u0000\u0000\u00a6\u0345\u0001\u0000\u0000\u0000\u00a8\u0347"+
+ "\u0001\u0000\u0000\u0000\u00aa\u0349\u0001\u0000\u0000\u0000\u00ac\u0356"+
+ "\u0001\u0000\u0000\u0000\u00ae\u036a\u0001\u0000\u0000\u0000\u00b0\u00b1"+
"\u0003\u0002\u0001\u0000\u00b1\u00b2\u0005\u0000\u0000\u0001\u00b2\u0001"+
"\u0001\u0000\u0000\u0000\u00b3\u00b4\u0006\u0001\uffff\uffff\u0000\u00b4"+
"\u00b5\u0003\u0004\u0002\u0000\u00b5\u00bb\u0001\u0000\u0000\u0000\u00b6"+
@@ -7787,7 +7863,7 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"\u00cb\u00e2\u0003*\u0015\u0000\u00cc\u00e2\u0003F#\u0000\u00cd\u00e2"+
"\u0003L&\u0000\u00ce\u00e2\u0003N\'\u0000\u00cf\u00e2\u0003R)\u0000\u00d0"+
"\u00e2\u0003Z-\u0000\u00d1\u00e2\u0003d2\u0000\u00d2\u00e2\u0003\\.\u0000"+
- "\u00d3\u00e2\u0003\u00a8T\u0000\u00d4\u00e2\u0003l6\u0000\u00d5\u00e2"+
+ "\u00d3\u00e2\u0003\u00aaU\u0000\u00d4\u00e2\u0003l6\u0000\u00d5\u00e2"+
"\u0003z=\u0000\u00d6\u00e2\u0003j5\u0000\u00d7\u00e2\u0003n7\u0000\u00d8"+
"\u00e2\u0003x<\u0000\u00d9\u00da\u0004\u0003\u0003\u0000\u00da\u00e2\u0003"+
"~?\u0000\u00db\u00dc\u0004\u0003\u0004\u0000\u00dc\u00e2\u0003|>\u0000"+
@@ -7899,7 +7975,7 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
">\u001f\u0000\u0191\u0193\u0003@ \u0000\u0192\u018f\u0001\u0000\u0000"+
"\u0000\u0192\u0190\u0001\u0000\u0000\u0000\u0192\u0191\u0001\u0000\u0000"+
"\u0000\u0193C\u0001\u0000\u0000\u0000\u0194\u0195\u0005\u000b\u0000\u0000"+
- "\u0195\u0196\u0003\u009aM\u0000\u0196E\u0001\u0000\u0000\u0000\u0197\u0198"+
+ "\u0195\u0196\u0003\u009cN\u0000\u0196E\u0001\u0000\u0000\u0000\u0197\u0198"+
"\u0005\u000f\u0000\u0000\u0198\u019d\u0003H$\u0000\u0199\u019a\u0005:"+
"\u0000\u0000\u019a\u019c\u0003H$\u0000\u019b\u0199\u0001\u0000\u0000\u0000"+
"\u019c\u019f\u0001\u0000\u0000\u0000\u019d\u019b\u0001\u0000\u0000\u0000"+
@@ -7922,18 +7998,18 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"\u01bd\u01be\u00034\u001a\u0000\u01be\u01c0\u0001\u0000\u0000\u0000\u01bf"+
"\u01b7\u0001\u0000\u0000\u0000\u01bf\u01bb\u0001\u0000\u0000\u0000\u01c0"+
"Q\u0001\u0000\u0000\u0000\u01c1\u01c2\u0005\b\u0000\u0000\u01c2\u01c3"+
- "\u0003\u008eG\u0000\u01c3\u01c5\u0003\u00a4R\u0000\u01c4\u01c6\u0003T"+
+ "\u0003\u0090H\u0000\u01c3\u01c5\u0003\u00a6S\u0000\u01c4\u01c6\u0003T"+
"*\u0000\u01c5\u01c4\u0001\u0000\u0000\u0000\u01c5\u01c6\u0001\u0000\u0000"+
"\u0000\u01c6S\u0001\u0000\u0000\u0000\u01c7\u01cc\u0003V+\u0000\u01c8"+
"\u01c9\u0005:\u0000\u0000\u01c9\u01cb\u0003V+\u0000\u01ca\u01c8\u0001"+
"\u0000\u0000\u0000\u01cb\u01ce\u0001\u0000\u0000\u0000\u01cc\u01ca\u0001"+
"\u0000\u0000\u0000\u01cc\u01cd\u0001\u0000\u0000\u0000\u01cdU\u0001\u0000"+
"\u0000\u0000\u01ce\u01cc\u0001\u0000\u0000\u0000\u01cf\u01d0\u0003:\u001d"+
- "\u0000\u01d0\u01d1\u00056\u0000\u0000\u01d1\u01d2\u0003\u009aM\u0000\u01d2"+
+ "\u0000\u01d0\u01d1\u00056\u0000\u0000\u01d1\u01d2\u0003\u009cN\u0000\u01d2"+
"W\u0001\u0000\u0000\u0000\u01d3\u01d4\u0005K\u0000\u0000\u01d4\u01d6\u0003"+
- "\u0094J\u0000\u01d5\u01d3\u0001\u0000\u0000\u0000\u01d5\u01d6\u0001\u0000"+
+ "\u0096K\u0000\u01d5\u01d3\u0001\u0000\u0000\u0000\u01d5\u01d6\u0001\u0000"+
"\u0000\u0000\u01d6Y\u0001\u0000\u0000\u0000\u01d7\u01d8\u0005\n\u0000"+
- "\u0000\u01d8\u01d9\u0003\u008eG\u0000\u01d9\u01da\u0003\u00a4R\u0000\u01da"+
+ "\u0000\u01d8\u01d9\u0003\u0090H\u0000\u01d9\u01da\u0003\u00a6S\u0000\u01da"+
"[\u0001\u0000\u0000\u0000\u01db\u01dc\u0005\u001c\u0000\u0000\u01dc\u01dd"+
"\u00030\u0018\u0000\u01dd]\u0001\u0000\u0000\u0000\u01de\u01df\u0005\u0006"+
"\u0000\u0000\u01df\u01e0\u0003`0\u0000\u01e0_\u0001\u0000\u0000\u0000"+
@@ -7954,7 +8030,7 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"\u0000\u01fd\u01ff\u0001\u0000\u0000\u0000\u01fe\u01fb\u0001\u0000\u0000"+
"\u0000\u01fe\u01ff\u0001\u0000\u0000\u0000\u01ff\u0200\u0001\u0000\u0000"+
"\u0000\u0200\u0201\u00034\u001a\u0000\u0201i\u0001\u0000\u0000\u0000\u0202"+
- "\u0203\u0005\u000e\u0000\u0000\u0203\u0204\u0003\u009aM\u0000\u0204k\u0001"+
+ "\u0203\u0005\u000e\u0000\u0000\u0203\u0204\u0003\u009cN\u0000\u0204k\u0001"+
"\u0000\u0000\u0000\u0205\u0206\u0005\u0004\u0000\u0000\u0206\u0209\u0003"+
"0\u0018\u0000\u0207\u0208\u0005F\u0000\u0000\u0208\u020a\u00030\u0018"+
"\u0000\u0209\u0207\u0001\u0000\u0000\u0000\u0209\u020a\u0001\u0000\u0000"+
@@ -7978,12 +8054,12 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"\u022c\u022d\u00030\u0018\u0000\u022d\u022e\u00056\u0000\u0000\u022e\u0230"+
"\u0001\u0000\u0000\u0000\u022f\u022c\u0001\u0000\u0000\u0000\u022f\u0230"+
"\u0001\u0000\u0000\u0000\u0230\u0231\u0001\u0000\u0000\u0000\u0231\u0232"+
- "\u0003\u009aM\u0000\u0232\u0233\u0005F\u0000\u0000\u0233\u0234\u0003\u0012"+
+ "\u0003\u009cN\u0000\u0232\u0233\u0005F\u0000\u0000\u0233\u0234\u0003\u0012"+
"\t\u0000\u0234\u0235\u0003X,\u0000\u0235y\u0001\u0000\u0000\u0000\u0236"+
"\u023a\u0005\u0007\u0000\u0000\u0237\u0238\u00030\u0018\u0000\u0238\u0239"+
"\u00056\u0000\u0000\u0239\u023b\u0001\u0000\u0000\u0000\u023a\u0237\u0001"+
"\u0000\u0000\u0000\u023a\u023b\u0001\u0000\u0000\u0000\u023b\u023c\u0001"+
- "\u0000\u0000\u0000\u023c\u023d\u0003\u008eG\u0000\u023d\u023e\u0003X,"+
+ "\u0000\u0000\u0000\u023c\u023d\u0003\u0090H\u0000\u023d\u023e\u0003X,"+
"\u0000\u023e{\u0001\u0000\u0000\u0000\u023f\u0240\u0005\u001b\u0000\u0000"+
"\u0240\u0241\u0003\u001c\u000e\u0000\u0241\u0242\u0005F\u0000\u0000\u0242"+
"\u0243\u00038\u001c\u0000\u0243}\u0001\u0000\u0000\u0000\u0244\u0245\u0005"+
@@ -8019,24 +8095,24 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"\u0000\u0000\u0278\u0276\u0001\u0000\u0000\u0000\u0279\u027b\u0003\u008a"+
"E\u0000\u027a\u027c\u0005C\u0000\u0000\u027b\u027a\u0001\u0000\u0000\u0000"+
"\u027b\u027c\u0001\u0000\u0000\u0000\u027c\u027d\u0001\u0000\u0000\u0000"+
- "\u027d\u027e\u0005B\u0000\u0000\u027e\u027f\u0003\u00a4R\u0000\u027f\u02a8"+
+ "\u027d\u027e\u0005B\u0000\u0000\u027e\u027f\u0003\u00a6S\u0000\u027f\u02a8"+
"\u0001\u0000\u0000\u0000\u0280\u0282\u0003\u008aE\u0000\u0281\u0283\u0005"+
"C\u0000\u0000\u0282\u0281\u0001\u0000\u0000\u0000\u0282\u0283\u0001\u0000"+
"\u0000\u0000\u0283\u0284\u0001\u0000\u0000\u0000\u0284\u0285\u0005I\u0000"+
- "\u0000\u0285\u0286\u0003\u00a4R\u0000\u0286\u02a8\u0001\u0000\u0000\u0000"+
+ "\u0000\u0285\u0286\u0003\u00a6S\u0000\u0286\u02a8\u0001\u0000\u0000\u0000"+
"\u0287\u0289\u0003\u008aE\u0000\u0288\u028a\u0005C\u0000\u0000\u0289\u0288"+
"\u0001\u0000\u0000\u0000\u0289\u028a\u0001\u0000\u0000\u0000\u028a\u028b"+
"\u0001\u0000\u0000\u0000\u028b\u028c\u0005B\u0000\u0000\u028c\u028d\u0005"+
- "_\u0000\u0000\u028d\u0292\u0003\u00a4R\u0000\u028e\u028f\u0005:\u0000"+
- "\u0000\u028f\u0291\u0003\u00a4R\u0000\u0290\u028e\u0001\u0000\u0000\u0000"+
+ "_\u0000\u0000\u028d\u0292\u0003\u00a6S\u0000\u028e\u028f\u0005:\u0000"+
+ "\u0000\u028f\u0291\u0003\u00a6S\u0000\u0290\u028e\u0001\u0000\u0000\u0000"+
"\u0291\u0294\u0001\u0000\u0000\u0000\u0292\u0290\u0001\u0000\u0000\u0000"+
"\u0292\u0293\u0001\u0000\u0000\u0000\u0293\u0295\u0001\u0000\u0000\u0000"+
"\u0294\u0292\u0001\u0000\u0000\u0000\u0295\u0296\u0005`\u0000\u0000\u0296"+
"\u02a8\u0001\u0000\u0000\u0000\u0297\u0299\u0003\u008aE\u0000\u0298\u029a"+
"\u0005C\u0000\u0000\u0299\u0298\u0001\u0000\u0000\u0000\u0299\u029a\u0001"+
"\u0000\u0000\u0000\u029a\u029b\u0001\u0000\u0000\u0000\u029b\u029c\u0005"+
- "I\u0000\u0000\u029c\u029d\u0005_\u0000\u0000\u029d\u02a2\u0003\u00a4R"+
- "\u0000\u029e\u029f\u0005:\u0000\u0000\u029f\u02a1\u0003\u00a4R\u0000\u02a0"+
+ "I\u0000\u0000\u029c\u029d\u0005_\u0000\u0000\u029d\u02a2\u0003\u00a6S"+
+ "\u0000\u029e\u029f\u0005:\u0000\u0000\u029f\u02a1\u0003\u00a6S\u0000\u02a0"+
"\u029e\u0001\u0000\u0000\u0000\u02a1\u02a4\u0001\u0000\u0000\u0000\u02a2"+
"\u02a0\u0001\u0000\u0000\u0000\u02a2\u02a3\u0001\u0000\u0000\u0000\u02a3"+
"\u02a5\u0001\u0000\u0000\u0000\u02a4\u02a2\u0001\u0000\u0000\u0000\u02a5"+
@@ -8046,114 +8122,121 @@ private boolean joinTarget_sempred(JoinTargetContext _localctx, int predIndex) {
"\u0001\u0000\u0000\u0000\u02a9\u02ac\u00030\u0018\u0000\u02aa\u02ab\u0005"+
"8\u0000\u0000\u02ab\u02ad\u0003\n\u0005\u0000\u02ac\u02aa\u0001\u0000"+
"\u0000\u0000\u02ac\u02ad\u0001\u0000\u0000\u0000\u02ad\u02ae\u0001\u0000"+
- "\u0000\u0000\u02ae\u02af\u00059\u0000\u0000\u02af\u02b0\u0003\u009aM\u0000"+
- "\u02b0\u0089\u0001\u0000\u0000\u0000\u02b1\u02b7\u0003\u008cF\u0000\u02b2"+
- "\u02b3\u0003\u008cF\u0000\u02b3\u02b4\u0003\u00a6S\u0000\u02b4\u02b5\u0003"+
- "\u008cF\u0000\u02b5\u02b7\u0001\u0000\u0000\u0000\u02b6\u02b1\u0001\u0000"+
- "\u0000\u0000\u02b6\u02b2\u0001\u0000\u0000\u0000\u02b7\u008b\u0001\u0000"+
- "\u0000\u0000\u02b8\u02b9\u0006F\uffff\uffff\u0000\u02b9\u02bd\u0003\u008e"+
- "G\u0000\u02ba\u02bb\u0007\u0005\u0000\u0000\u02bb\u02bd\u0003\u008cF\u0003"+
- "\u02bc\u02b8\u0001\u0000\u0000\u0000\u02bc\u02ba\u0001\u0000\u0000\u0000"+
- "\u02bd\u02c6\u0001\u0000\u0000\u0000\u02be\u02bf\n\u0002\u0000\u0000\u02bf"+
- "\u02c0\u0007\u0006\u0000\u0000\u02c0\u02c5\u0003\u008cF\u0003\u02c1\u02c2"+
- "\n\u0001\u0000\u0000\u02c2\u02c3\u0007\u0005\u0000\u0000\u02c3\u02c5\u0003"+
- "\u008cF\u0002\u02c4\u02be\u0001\u0000\u0000\u0000\u02c4\u02c1\u0001\u0000"+
- "\u0000\u0000\u02c5\u02c8\u0001\u0000\u0000\u0000\u02c6\u02c4\u0001\u0000"+
- "\u0000\u0000\u02c6\u02c7\u0001\u0000\u0000\u0000\u02c7\u008d\u0001\u0000"+
- "\u0000\u0000\u02c8\u02c6\u0001\u0000\u0000\u0000\u02c9\u02ca\u0006G\uffff"+
- "\uffff\u0000\u02ca\u02d2\u0003\u009aM\u0000\u02cb\u02d2\u00030\u0018\u0000"+
- "\u02cc\u02d2\u0003\u0090H\u0000\u02cd\u02ce\u0005_\u0000\u0000\u02ce\u02cf"+
- "\u0003\u0084B\u0000\u02cf\u02d0\u0005`\u0000\u0000\u02d0\u02d2\u0001\u0000"+
- "\u0000\u0000\u02d1\u02c9\u0001\u0000\u0000\u0000\u02d1\u02cb\u0001\u0000"+
- "\u0000\u0000\u02d1\u02cc\u0001\u0000\u0000\u0000\u02d1\u02cd\u0001\u0000"+
- "\u0000\u0000\u02d2\u02d8\u0001\u0000\u0000\u0000\u02d3\u02d4\n\u0001\u0000"+
- "\u0000\u02d4\u02d5\u00058\u0000\u0000\u02d5\u02d7\u0003\n\u0005\u0000"+
- "\u02d6\u02d3\u0001\u0000\u0000\u0000\u02d7\u02da\u0001\u0000\u0000\u0000"+
- "\u02d8\u02d6\u0001\u0000\u0000\u0000\u02d8\u02d9\u0001\u0000\u0000\u0000"+
- "\u02d9\u008f\u0001\u0000\u0000\u0000\u02da\u02d8\u0001\u0000\u0000\u0000"+
- "\u02db\u02dc\u0003\u0092I\u0000\u02dc\u02ea\u0005_\u0000\u0000\u02dd\u02eb"+
- "\u0005U\u0000\u0000\u02de\u02e3\u0003\u0084B\u0000\u02df\u02e0\u0005:"+
- "\u0000\u0000\u02e0\u02e2\u0003\u0084B\u0000\u02e1\u02df\u0001\u0000\u0000"+
- "\u0000\u02e2\u02e5\u0001\u0000\u0000\u0000\u02e3\u02e1\u0001\u0000\u0000"+
- "\u0000\u02e3\u02e4\u0001\u0000\u0000\u0000\u02e4\u02e8\u0001\u0000\u0000"+
- "\u0000\u02e5\u02e3\u0001\u0000\u0000\u0000\u02e6\u02e7\u0005:\u0000\u0000"+
- "\u02e7\u02e9\u0003\u0094J\u0000\u02e8\u02e6\u0001\u0000\u0000\u0000\u02e8"+
- "\u02e9\u0001\u0000\u0000\u0000\u02e9\u02eb\u0001\u0000\u0000\u0000\u02ea"+
- "\u02dd\u0001\u0000\u0000\u0000\u02ea\u02de\u0001\u0000\u0000\u0000\u02ea"+
- "\u02eb\u0001\u0000\u0000\u0000\u02eb\u02ec\u0001\u0000\u0000\u0000\u02ec"+
- "\u02ed\u0005`\u0000\u0000\u02ed\u0091\u0001\u0000\u0000\u0000\u02ee\u02f2"+
- "\u0003B!\u0000\u02ef\u02f2\u0005>\u0000\u0000\u02f0\u02f2\u0005A\u0000"+
- "\u0000\u02f1\u02ee\u0001\u0000\u0000\u0000\u02f1\u02ef\u0001\u0000\u0000"+
- "\u0000\u02f1\u02f0\u0001\u0000\u0000\u0000\u02f2\u0093\u0001\u0000\u0000"+
- "\u0000\u02f3\u02fc\u0005X\u0000\u0000\u02f4\u02f9\u0003\u0096K\u0000\u02f5"+
- "\u02f6\u0005:\u0000\u0000\u02f6\u02f8\u0003\u0096K\u0000\u02f7\u02f5\u0001"+
- "\u0000\u0000\u0000\u02f8\u02fb\u0001\u0000\u0000\u0000\u02f9\u02f7\u0001"+
- "\u0000\u0000\u0000\u02f9\u02fa\u0001\u0000\u0000\u0000\u02fa\u02fd\u0001"+
- "\u0000\u0000\u0000\u02fb\u02f9\u0001\u0000\u0000\u0000\u02fc\u02f4\u0001"+
- "\u0000\u0000\u0000\u02fc\u02fd\u0001\u0000\u0000\u0000\u02fd\u02fe\u0001"+
- "\u0000\u0000\u0000\u02fe\u02ff\u0005Y\u0000\u0000\u02ff\u0095\u0001\u0000"+
- "\u0000\u0000\u0300\u0301\u0003\u00a4R\u0000\u0301\u0302\u00059\u0000\u0000"+
- "\u0302\u0303\u0003\u0098L\u0000\u0303\u0097\u0001\u0000\u0000\u0000\u0304"+
- "\u0307\u0003\u009aM\u0000\u0305\u0307\u0003\u0094J\u0000\u0306\u0304\u0001"+
- "\u0000\u0000\u0000\u0306\u0305\u0001\u0000\u0000\u0000\u0307\u0099\u0001"+
- "\u0000\u0000\u0000\u0308\u0333\u0005D\u0000\u0000\u0309\u030a\u0003\u00a2"+
- "Q\u0000\u030a\u030b\u0005a\u0000\u0000\u030b\u0333\u0001\u0000\u0000\u0000"+
- "\u030c\u0333\u0003\u00a0P\u0000\u030d\u0333\u0003\u00a2Q\u0000\u030e\u0333"+
- "\u0003\u009cN\u0000\u030f\u0333\u0003>\u001f\u0000\u0310\u0333\u0003\u00a4"+
- "R\u0000\u0311\u0312\u0005]\u0000\u0000\u0312\u0317\u0003\u009eO\u0000"+
- "\u0313\u0314\u0005:\u0000\u0000\u0314\u0316\u0003\u009eO\u0000\u0315\u0313"+
- "\u0001\u0000\u0000\u0000\u0316\u0319\u0001\u0000\u0000\u0000\u0317\u0315"+
- "\u0001\u0000\u0000\u0000\u0317\u0318\u0001\u0000\u0000\u0000\u0318\u031a"+
- "\u0001\u0000\u0000\u0000\u0319\u0317\u0001\u0000\u0000\u0000\u031a\u031b"+
- "\u0005^\u0000\u0000\u031b\u0333\u0001\u0000\u0000\u0000\u031c\u031d\u0005"+
- "]\u0000\u0000\u031d\u0322\u0003\u009cN\u0000\u031e\u031f\u0005:\u0000"+
- "\u0000\u031f\u0321\u0003\u009cN\u0000\u0320\u031e\u0001\u0000\u0000\u0000"+
- "\u0321\u0324\u0001\u0000\u0000\u0000\u0322\u0320\u0001\u0000\u0000\u0000"+
- "\u0322\u0323\u0001\u0000\u0000\u0000\u0323\u0325\u0001\u0000\u0000\u0000"+
- "\u0324\u0322\u0001\u0000\u0000\u0000\u0325\u0326\u0005^\u0000\u0000\u0326"+
- "\u0333\u0001\u0000\u0000\u0000\u0327\u0328\u0005]\u0000\u0000\u0328\u032d"+
- "\u0003\u00a4R\u0000\u0329\u032a\u0005:\u0000\u0000\u032a\u032c\u0003\u00a4"+
- "R\u0000\u032b\u0329\u0001\u0000\u0000\u0000\u032c\u032f\u0001\u0000\u0000"+
- "\u0000\u032d\u032b\u0001\u0000\u0000\u0000\u032d\u032e\u0001\u0000\u0000"+
- "\u0000\u032e\u0330\u0001\u0000\u0000\u0000\u032f\u032d\u0001\u0000\u0000"+
- "\u0000\u0330\u0331\u0005^\u0000\u0000\u0331\u0333\u0001\u0000\u0000\u0000"+
- "\u0332\u0308\u0001\u0000\u0000\u0000\u0332\u0309\u0001\u0000\u0000\u0000"+
- "\u0332\u030c\u0001\u0000\u0000\u0000\u0332\u030d\u0001\u0000\u0000\u0000"+
- "\u0332\u030e\u0001\u0000\u0000\u0000\u0332\u030f\u0001\u0000\u0000\u0000"+
- "\u0332\u0310\u0001\u0000\u0000\u0000\u0332\u0311\u0001\u0000\u0000\u0000"+
- "\u0332\u031c\u0001\u0000\u0000\u0000\u0332\u0327\u0001\u0000\u0000\u0000"+
- "\u0333\u009b\u0001\u0000\u0000\u0000\u0334\u0335\u0007\u0007\u0000\u0000"+
- "\u0335\u009d\u0001\u0000\u0000\u0000\u0336\u0339\u0003\u00a0P\u0000\u0337"+
- "\u0339\u0003\u00a2Q\u0000\u0338\u0336\u0001\u0000\u0000\u0000\u0338\u0337"+
- "\u0001\u0000\u0000\u0000\u0339\u009f\u0001\u0000\u0000\u0000\u033a\u033c"+
- "\u0007\u0005\u0000\u0000\u033b\u033a\u0001\u0000\u0000\u0000\u033b\u033c"+
- "\u0001\u0000\u0000\u0000\u033c\u033d\u0001\u0000\u0000\u0000\u033d\u033e"+
- "\u00053\u0000\u0000\u033e\u00a1\u0001\u0000\u0000\u0000\u033f\u0341\u0007"+
- "\u0005\u0000\u0000\u0340\u033f\u0001\u0000\u0000\u0000\u0340\u0341\u0001"+
- "\u0000\u0000\u0000\u0341\u0342\u0001\u0000\u0000\u0000\u0342\u0343\u0005"+
- "2\u0000\u0000\u0343\u00a3\u0001\u0000\u0000\u0000\u0344\u0345\u00051\u0000"+
- "\u0000\u0345\u00a5\u0001\u0000\u0000\u0000\u0346\u0347\u0007\b\u0000\u0000"+
- "\u0347\u00a7\u0001\u0000\u0000\u0000\u0348\u0349\u0007\t\u0000\u0000\u0349"+
- "\u034a\u0005n\u0000\u0000\u034a\u034b\u0003\u00aaU\u0000\u034b\u034c\u0003"+
- "\u00acV\u0000\u034c\u00a9\u0001\u0000\u0000\u0000\u034d\u034e\u0004U\u000f"+
- "\u0000\u034e\u0350\u0003\u001c\u000e\u0000\u034f\u0351\u0005\u0080\u0000"+
- "\u0000\u0350\u034f\u0001\u0000\u0000\u0000\u0350\u0351\u0001\u0000\u0000"+
- "\u0000\u0351\u0352\u0001\u0000\u0000\u0000\u0352\u0353\u0005g\u0000\u0000"+
- "\u0353\u0356\u0001\u0000\u0000\u0000\u0354\u0356\u0003\u001c\u000e\u0000"+
- "\u0355\u034d\u0001\u0000\u0000\u0000\u0355\u0354\u0001\u0000\u0000\u0000"+
- "\u0356\u00ab\u0001\u0000\u0000\u0000\u0357\u0358\u0005F\u0000\u0000\u0358"+
- "\u035d\u0003\u00aeW\u0000\u0359\u035a\u0005:\u0000\u0000\u035a\u035c\u0003"+
- "\u00aeW\u0000\u035b\u0359\u0001\u0000\u0000\u0000\u035c\u035f\u0001\u0000"+
- "\u0000\u0000\u035d\u035b\u0001\u0000\u0000\u0000\u035d\u035e\u0001\u0000"+
- "\u0000\u0000\u035e\u00ad\u0001\u0000\u0000\u0000\u035f\u035d\u0001\u0000"+
- "\u0000\u0000\u0360\u0361\u0003\u008aE\u0000\u0361\u00af\u0001\u0000\u0000"+
- "\u0000R\u00bb\u00c5\u00e1\u00f0\u00f6\u00ff\u0105\u0112\u0116\u0121\u0131"+
+ "\u0000\u0000\u02ae\u02af\u00059\u0000\u0000\u02af\u02b0\u0003\u009cN\u0000"+
+ "\u02b0\u0089\u0001\u0000\u0000\u0000\u02b1\u02b4\u0003\u008eG\u0000\u02b2"+
+ "\u02b4\u0003\u008cF\u0000\u02b3\u02b1\u0001\u0000\u0000\u0000\u02b3\u02b2"+
+ "\u0001\u0000\u0000\u0000\u02b4\u008b\u0001\u0000\u0000\u0000\u02b5\u02b6"+
+ "\u0003\u008eG\u0000\u02b6\u02b7\u0003\u00a8T\u0000\u02b7\u02b8\u0003\u008e"+
+ "G\u0000\u02b8\u008d\u0001\u0000\u0000\u0000\u02b9\u02ba\u0006G\uffff\uffff"+
+ "\u0000\u02ba\u02be\u0003\u0090H\u0000\u02bb\u02bc\u0007\u0005\u0000\u0000"+
+ "\u02bc\u02be\u0003\u008eG\u0003\u02bd\u02b9\u0001\u0000\u0000\u0000\u02bd"+
+ "\u02bb\u0001\u0000\u0000\u0000\u02be\u02c7\u0001\u0000\u0000\u0000\u02bf"+
+ "\u02c0\n\u0002\u0000\u0000\u02c0\u02c1\u0007\u0006\u0000\u0000\u02c1\u02c6"+
+ "\u0003\u008eG\u0003\u02c2\u02c3\n\u0001\u0000\u0000\u02c3\u02c4\u0007"+
+ "\u0005\u0000\u0000\u02c4\u02c6\u0003\u008eG\u0002\u02c5\u02bf\u0001\u0000"+
+ "\u0000\u0000\u02c5\u02c2\u0001\u0000\u0000\u0000\u02c6\u02c9\u0001\u0000"+
+ "\u0000\u0000\u02c7\u02c5\u0001\u0000\u0000\u0000\u02c7\u02c8\u0001\u0000"+
+ "\u0000\u0000\u02c8\u008f\u0001\u0000\u0000\u0000\u02c9\u02c7\u0001\u0000"+
+ "\u0000\u0000\u02ca\u02cb\u0006H\uffff\uffff\u0000\u02cb\u02d3\u0003\u009c"+
+ "N\u0000\u02cc\u02d3\u00030\u0018\u0000\u02cd\u02d3\u0003\u0092I\u0000"+
+ "\u02ce\u02cf\u0005_\u0000\u0000\u02cf\u02d0\u0003\u0084B\u0000\u02d0\u02d1"+
+ "\u0005`\u0000\u0000\u02d1\u02d3\u0001\u0000\u0000\u0000\u02d2\u02ca\u0001"+
+ "\u0000\u0000\u0000\u02d2\u02cc\u0001\u0000\u0000\u0000\u02d2\u02cd\u0001"+
+ "\u0000\u0000\u0000\u02d2\u02ce\u0001\u0000\u0000\u0000\u02d3\u02d9\u0001"+
+ "\u0000\u0000\u0000\u02d4\u02d5\n\u0001\u0000\u0000\u02d5\u02d6\u00058"+
+ "\u0000\u0000\u02d6\u02d8\u0003\n\u0005\u0000\u02d7\u02d4\u0001\u0000\u0000"+
+ "\u0000\u02d8\u02db\u0001\u0000\u0000\u0000\u02d9\u02d7\u0001\u0000\u0000"+
+ "\u0000\u02d9\u02da\u0001\u0000\u0000\u0000\u02da\u0091\u0001\u0000\u0000"+
+ "\u0000\u02db\u02d9\u0001\u0000\u0000\u0000\u02dc\u02dd\u0003\u0094J\u0000"+
+ "\u02dd\u02eb\u0005_\u0000\u0000\u02de\u02ec\u0005U\u0000\u0000\u02df\u02e4"+
+ "\u0003\u0084B\u0000\u02e0\u02e1\u0005:\u0000\u0000\u02e1\u02e3\u0003\u0084"+
+ "B\u0000\u02e2\u02e0\u0001\u0000\u0000\u0000\u02e3\u02e6\u0001\u0000\u0000"+
+ "\u0000\u02e4\u02e2\u0001\u0000\u0000\u0000\u02e4\u02e5\u0001\u0000\u0000"+
+ "\u0000\u02e5\u02e9\u0001\u0000\u0000\u0000\u02e6\u02e4\u0001\u0000\u0000"+
+ "\u0000\u02e7\u02e8\u0005:\u0000\u0000\u02e8\u02ea\u0003\u0096K\u0000\u02e9"+
+ "\u02e7\u0001\u0000\u0000\u0000\u02e9\u02ea\u0001\u0000\u0000\u0000\u02ea"+
+ "\u02ec\u0001\u0000\u0000\u0000\u02eb\u02de\u0001\u0000\u0000\u0000\u02eb"+
+ "\u02df\u0001\u0000\u0000\u0000\u02eb\u02ec\u0001\u0000\u0000\u0000\u02ec"+
+ "\u02ed\u0001\u0000\u0000\u0000\u02ed\u02ee\u0005`\u0000\u0000\u02ee\u0093"+
+ "\u0001\u0000\u0000\u0000\u02ef\u02f3\u0003B!\u0000\u02f0\u02f3\u0005>"+
+ "\u0000\u0000\u02f1\u02f3\u0005A\u0000\u0000\u02f2\u02ef\u0001\u0000\u0000"+
+ "\u0000\u02f2\u02f0\u0001\u0000\u0000\u0000\u02f2\u02f1\u0001\u0000\u0000"+
+ "\u0000\u02f3\u0095\u0001\u0000\u0000\u0000\u02f4\u02fd\u0005X\u0000\u0000"+
+ "\u02f5\u02fa\u0003\u0098L\u0000\u02f6\u02f7\u0005:\u0000\u0000\u02f7\u02f9"+
+ "\u0003\u0098L\u0000\u02f8\u02f6\u0001\u0000\u0000\u0000\u02f9\u02fc\u0001"+
+ "\u0000\u0000\u0000\u02fa\u02f8\u0001\u0000\u0000\u0000\u02fa\u02fb\u0001"+
+ "\u0000\u0000\u0000\u02fb\u02fe\u0001\u0000\u0000\u0000\u02fc\u02fa\u0001"+
+ "\u0000\u0000\u0000\u02fd\u02f5\u0001\u0000\u0000\u0000\u02fd\u02fe\u0001"+
+ "\u0000\u0000\u0000\u02fe\u02ff\u0001\u0000\u0000\u0000\u02ff\u0300\u0005"+
+ "Y\u0000\u0000\u0300\u0097\u0001\u0000\u0000\u0000\u0301\u0302\u0003\u00a6"+
+ "S\u0000\u0302\u0303\u00059\u0000\u0000\u0303\u0304\u0003\u009aM\u0000"+
+ "\u0304\u0099\u0001\u0000\u0000\u0000\u0305\u0308\u0003\u009cN\u0000\u0306"+
+ "\u0308\u0003\u0096K\u0000\u0307\u0305\u0001\u0000\u0000\u0000\u0307\u0306"+
+ "\u0001\u0000\u0000\u0000\u0308\u009b\u0001\u0000\u0000\u0000\u0309\u0334"+
+ "\u0005D\u0000\u0000\u030a\u030b\u0003\u00a4R\u0000\u030b\u030c\u0005a"+
+ "\u0000\u0000\u030c\u0334\u0001\u0000\u0000\u0000\u030d\u0334\u0003\u00a2"+
+ "Q\u0000\u030e\u0334\u0003\u00a4R\u0000\u030f\u0334\u0003\u009eO\u0000"+
+ "\u0310\u0334\u0003>\u001f\u0000\u0311\u0334\u0003\u00a6S\u0000\u0312\u0313"+
+ "\u0005]\u0000\u0000\u0313\u0318\u0003\u00a0P\u0000\u0314\u0315\u0005:"+
+ "\u0000\u0000\u0315\u0317\u0003\u00a0P\u0000\u0316\u0314\u0001\u0000\u0000"+
+ "\u0000\u0317\u031a\u0001\u0000\u0000\u0000\u0318\u0316\u0001\u0000\u0000"+
+ "\u0000\u0318\u0319\u0001\u0000\u0000\u0000\u0319\u031b\u0001\u0000\u0000"+
+ "\u0000\u031a\u0318\u0001\u0000\u0000\u0000\u031b\u031c\u0005^\u0000\u0000"+
+ "\u031c\u0334\u0001\u0000\u0000\u0000\u031d\u031e\u0005]\u0000\u0000\u031e"+
+ "\u0323\u0003\u009eO\u0000\u031f\u0320\u0005:\u0000\u0000\u0320\u0322\u0003"+
+ "\u009eO\u0000\u0321\u031f\u0001\u0000\u0000\u0000\u0322\u0325\u0001\u0000"+
+ "\u0000\u0000\u0323\u0321\u0001\u0000\u0000\u0000\u0323\u0324\u0001\u0000"+
+ "\u0000\u0000\u0324\u0326\u0001\u0000\u0000\u0000\u0325\u0323\u0001\u0000"+
+ "\u0000\u0000\u0326\u0327\u0005^\u0000\u0000\u0327\u0334\u0001\u0000\u0000"+
+ "\u0000\u0328\u0329\u0005]\u0000\u0000\u0329\u032e\u0003\u00a6S\u0000\u032a"+
+ "\u032b\u0005:\u0000\u0000\u032b\u032d\u0003\u00a6S\u0000\u032c\u032a\u0001"+
+ "\u0000\u0000\u0000\u032d\u0330\u0001\u0000\u0000\u0000\u032e\u032c\u0001"+
+ "\u0000\u0000\u0000\u032e\u032f\u0001\u0000\u0000\u0000\u032f\u0331\u0001"+
+ "\u0000\u0000\u0000\u0330\u032e\u0001\u0000\u0000\u0000\u0331\u0332\u0005"+
+ "^\u0000\u0000\u0332\u0334\u0001\u0000\u0000\u0000\u0333\u0309\u0001\u0000"+
+ "\u0000\u0000\u0333\u030a\u0001\u0000\u0000\u0000\u0333\u030d\u0001\u0000"+
+ "\u0000\u0000\u0333\u030e\u0001\u0000\u0000\u0000\u0333\u030f\u0001\u0000"+
+ "\u0000\u0000\u0333\u0310\u0001\u0000\u0000\u0000\u0333\u0311\u0001\u0000"+
+ "\u0000\u0000\u0333\u0312\u0001\u0000\u0000\u0000\u0333\u031d\u0001\u0000"+
+ "\u0000\u0000\u0333\u0328\u0001\u0000\u0000\u0000\u0334\u009d\u0001\u0000"+
+ "\u0000\u0000\u0335\u0336\u0007\u0007\u0000\u0000\u0336\u009f\u0001\u0000"+
+ "\u0000\u0000\u0337\u033a\u0003\u00a2Q\u0000\u0338\u033a\u0003\u00a4R\u0000"+
+ "\u0339\u0337\u0001\u0000\u0000\u0000\u0339\u0338\u0001\u0000\u0000\u0000"+
+ "\u033a\u00a1\u0001\u0000\u0000\u0000\u033b\u033d\u0007\u0005\u0000\u0000"+
+ "\u033c\u033b\u0001\u0000\u0000\u0000\u033c\u033d\u0001\u0000\u0000\u0000"+
+ "\u033d\u033e\u0001\u0000\u0000\u0000\u033e\u033f\u00053\u0000\u0000\u033f"+
+ "\u00a3\u0001\u0000\u0000\u0000\u0340\u0342\u0007\u0005\u0000\u0000\u0341"+
+ "\u0340\u0001\u0000\u0000\u0000\u0341\u0342\u0001\u0000\u0000\u0000\u0342"+
+ "\u0343\u0001\u0000\u0000\u0000\u0343\u0344\u00052\u0000\u0000\u0344\u00a5"+
+ "\u0001\u0000\u0000\u0000\u0345\u0346\u00051\u0000\u0000\u0346\u00a7\u0001"+
+ "\u0000\u0000\u0000\u0347\u0348\u0007\b\u0000\u0000\u0348\u00a9\u0001\u0000"+
+ "\u0000\u0000\u0349\u034a\u0007\t\u0000\u0000\u034a\u034b\u0005n\u0000"+
+ "\u0000\u034b\u034c\u0003\u00acV\u0000\u034c\u034d\u0003\u00aeW\u0000\u034d"+
+ "\u00ab\u0001\u0000\u0000\u0000\u034e\u034f\u0004V\u000f\u0000\u034f\u0351"+
+ "\u0003\u001c\u000e\u0000\u0350\u0352\u0005\u0080\u0000\u0000\u0351\u0350"+
+ "\u0001\u0000\u0000\u0000\u0351\u0352\u0001\u0000\u0000\u0000\u0352\u0353"+
+ "\u0001\u0000\u0000\u0000\u0353\u0354\u0005g\u0000\u0000\u0354\u0357\u0001"+
+ "\u0000\u0000\u0000\u0355\u0357\u0003\u001c\u000e\u0000\u0356\u034e\u0001"+
+ "\u0000\u0000\u0000\u0356\u0355\u0001\u0000\u0000\u0000\u0357\u00ad\u0001"+
+ "\u0000\u0000\u0000\u0358\u0359\u0005F\u0000\u0000\u0359\u035e\u00030\u0018"+
+ "\u0000\u035a\u035b\u0005:\u0000\u0000\u035b\u035d\u00030\u0018\u0000\u035c"+
+ "\u035a\u0001\u0000\u0000\u0000\u035d\u0360\u0001\u0000\u0000\u0000\u035e"+
+ "\u035c\u0001\u0000\u0000\u0000\u035e\u035f\u0001\u0000\u0000\u0000\u035f"+
+ "\u036b\u0001\u0000\u0000\u0000\u0360\u035e\u0001\u0000\u0000\u0000\u0361"+
+ "\u0362\u0005F\u0000\u0000\u0362\u0367\u0003\u008cF\u0000\u0363\u0364\u0005"+
+ "4\u0000\u0000\u0364\u0366\u0003\u008cF\u0000\u0365\u0363\u0001\u0000\u0000"+
+ "\u0000\u0366\u0369\u0001\u0000\u0000\u0000\u0367\u0365\u0001\u0000\u0000"+
+ "\u0000\u0367\u0368\u0001\u0000\u0000\u0000\u0368\u036b\u0001\u0000\u0000"+
+ "\u0000\u0369\u0367\u0001\u0000\u0000\u0000\u036a\u0358\u0001\u0000\u0000"+
+ "\u0000\u036a\u0361\u0001\u0000\u0000\u0000\u036b\u00af\u0001\u0000\u0000"+
+ "\u0000T\u00bb\u00c5\u00e1\u00f0\u00f6\u00ff\u0105\u0112\u0116\u0121\u0131"+
"\u0139\u013d\u0144\u014a\u014f\u0158\u015f\u0165\u016e\u0175\u017d\u0185"+
"\u0189\u018d\u0192\u019d\u01a2\u01a6\u01b4\u01bf\u01c5\u01cc\u01d5\u01ec"+
"\u01f4\u01f7\u01fe\u0209\u0210\u0218\u0226\u022f\u023a\u0248\u0256\u025f"+
"\u0267\u026c\u0274\u0276\u027b\u0282\u0289\u0292\u0299\u02a2\u02a7\u02ac"+
- "\u02b6\u02bc\u02c4\u02c6\u02d1\u02d8\u02e3\u02e8\u02ea\u02f1\u02f9\u02fc"+
- "\u0306\u0317\u0322\u032d\u0332\u0338\u033b\u0340\u0350\u0355\u035d";
+ "\u02b3\u02bd\u02c5\u02c7\u02d2\u02d9\u02e4\u02e9\u02eb\u02f2\u02fa\u02fd"+
+ "\u0307\u0318\u0323\u032e\u0333\u0339\u033c\u0341\u0351\u0356\u035e\u0367"+
+ "\u036a";
public static final ATN _ATN =
new ATNDeserializer().deserialize(_serializedATN.toCharArray());
static {
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java
index d54fc58ab4261..a2c1d8e7db563 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseListener.java
@@ -1028,6 +1028,18 @@ public class EsqlBaseParserBaseListener implements EsqlBaseParserListener {
* The default implementation does nothing.
*/
@Override public void exitComparison(EsqlBaseParser.ComparisonContext ctx) { }
+ /**
+ * {@inheritDoc}
+ *
+ * The default implementation does nothing.
+ */
+ @Override public void enterComparisonExpression(EsqlBaseParser.ComparisonExpressionContext ctx) { }
+ /**
+ * {@inheritDoc}
+ *
+ * The default implementation does nothing.
+ */
+ @Override public void exitComparisonExpression(EsqlBaseParser.ComparisonExpressionContext ctx) { }
/**
* {@inheritDoc}
*
@@ -1405,25 +1417,25 @@ public class EsqlBaseParserBaseListener implements EsqlBaseParserListener {
*
* The default implementation does nothing.
*/
- @Override public void enterJoinCondition(EsqlBaseParser.JoinConditionContext ctx) { }
+ @Override public void enterFieldBasedLookupJoin(EsqlBaseParser.FieldBasedLookupJoinContext ctx) { }
/**
* {@inheritDoc}
*
* The default implementation does nothing.
*/
- @Override public void exitJoinCondition(EsqlBaseParser.JoinConditionContext ctx) { }
+ @Override public void exitFieldBasedLookupJoin(EsqlBaseParser.FieldBasedLookupJoinContext ctx) { }
/**
* {@inheritDoc}
*
* The default implementation does nothing.
*/
- @Override public void enterJoinPredicate(EsqlBaseParser.JoinPredicateContext ctx) { }
+ @Override public void enterExpressionBasedLookupJoin(EsqlBaseParser.ExpressionBasedLookupJoinContext ctx) { }
/**
* {@inheritDoc}
*
* The default implementation does nothing.
*/
- @Override public void exitJoinPredicate(EsqlBaseParser.JoinPredicateContext ctx) { }
+ @Override public void exitExpressionBasedLookupJoin(EsqlBaseParser.ExpressionBasedLookupJoinContext ctx) { }
/**
* {@inheritDoc}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java
index 432336ef7a05f..2d8f85fd0a6a1 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserBaseVisitor.java
@@ -608,6 +608,13 @@ public class EsqlBaseParserBaseVisitor extends AbstractParseTreeVisitor im
* {@link #visitChildren} on {@code ctx}.
*/
@Override public T visitComparison(EsqlBaseParser.ComparisonContext ctx) { return visitChildren(ctx); }
+ /**
+ * {@inheritDoc}
+ *
+ * The default implementation returns the result of calling
+ * {@link #visitChildren} on {@code ctx}.
+ */
+ @Override public T visitComparisonExpression(EsqlBaseParser.ComparisonExpressionContext ctx) { return visitChildren(ctx); }
/**
* {@inheritDoc}
*
@@ -831,12 +838,12 @@ public class EsqlBaseParserBaseVisitor extends AbstractParseTreeVisitor im
* The default implementation returns the result of calling
* {@link #visitChildren} on {@code ctx}.
*/
- @Override public T visitJoinCondition(EsqlBaseParser.JoinConditionContext ctx) { return visitChildren(ctx); }
+ @Override public T visitFieldBasedLookupJoin(EsqlBaseParser.FieldBasedLookupJoinContext ctx) { return visitChildren(ctx); }
/**
* {@inheritDoc}
*
* The default implementation returns the result of calling
* {@link #visitChildren} on {@code ctx}.
*/
- @Override public T visitJoinPredicate(EsqlBaseParser.JoinPredicateContext ctx) { return visitChildren(ctx); }
+ @Override public T visitExpressionBasedLookupJoin(EsqlBaseParser.ExpressionBasedLookupJoinContext ctx) { return visitChildren(ctx); }
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java
index 6ba930fb79419..49bb4302d0352 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserListener.java
@@ -901,6 +901,16 @@ public interface EsqlBaseParserListener extends ParseTreeListener {
* @param ctx the parse tree
*/
void exitComparison(EsqlBaseParser.ComparisonContext ctx);
+ /**
+ * Enter a parse tree produced by {@link EsqlBaseParser#comparisonExpression}.
+ * @param ctx the parse tree
+ */
+ void enterComparisonExpression(EsqlBaseParser.ComparisonExpressionContext ctx);
+ /**
+ * Exit a parse tree produced by {@link EsqlBaseParser#comparisonExpression}.
+ * @param ctx the parse tree
+ */
+ void exitComparisonExpression(EsqlBaseParser.ComparisonExpressionContext ctx);
/**
* Enter a parse tree produced by the {@code operatorExpressionDefault}
* labeled alternative in {@link EsqlBaseParser#operatorExpression}.
@@ -1248,23 +1258,27 @@ public interface EsqlBaseParserListener extends ParseTreeListener {
*/
void exitJoinTarget(EsqlBaseParser.JoinTargetContext ctx);
/**
- * Enter a parse tree produced by {@link EsqlBaseParser#joinCondition}.
+ * Enter a parse tree produced by the {@code fieldBasedLookupJoin}
+ * labeled alternative in {@link EsqlBaseParser#joinCondition}.
* @param ctx the parse tree
*/
- void enterJoinCondition(EsqlBaseParser.JoinConditionContext ctx);
+ void enterFieldBasedLookupJoin(EsqlBaseParser.FieldBasedLookupJoinContext ctx);
/**
- * Exit a parse tree produced by {@link EsqlBaseParser#joinCondition}.
+ * Exit a parse tree produced by the {@code fieldBasedLookupJoin}
+ * labeled alternative in {@link EsqlBaseParser#joinCondition}.
* @param ctx the parse tree
*/
- void exitJoinCondition(EsqlBaseParser.JoinConditionContext ctx);
+ void exitFieldBasedLookupJoin(EsqlBaseParser.FieldBasedLookupJoinContext ctx);
/**
- * Enter a parse tree produced by {@link EsqlBaseParser#joinPredicate}.
+ * Enter a parse tree produced by the {@code expressionBasedLookupJoin}
+ * labeled alternative in {@link EsqlBaseParser#joinCondition}.
* @param ctx the parse tree
*/
- void enterJoinPredicate(EsqlBaseParser.JoinPredicateContext ctx);
+ void enterExpressionBasedLookupJoin(EsqlBaseParser.ExpressionBasedLookupJoinContext ctx);
/**
- * Exit a parse tree produced by {@link EsqlBaseParser#joinPredicate}.
+ * Exit a parse tree produced by the {@code expressionBasedLookupJoin}
+ * labeled alternative in {@link EsqlBaseParser#joinCondition}.
* @param ctx the parse tree
*/
- void exitJoinPredicate(EsqlBaseParser.JoinPredicateContext ctx);
+ void exitExpressionBasedLookupJoin(EsqlBaseParser.ExpressionBasedLookupJoinContext ctx);
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java
index 683629cf94e19..7c2c4d4c8f306 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlBaseParserVisitor.java
@@ -545,6 +545,12 @@ public interface EsqlBaseParserVisitor extends ParseTreeVisitor {
* @return the visitor result
*/
T visitComparison(EsqlBaseParser.ComparisonContext ctx);
+ /**
+ * Visit a parse tree produced by {@link EsqlBaseParser#comparisonExpression}.
+ * @param ctx the parse tree
+ * @return the visitor result
+ */
+ T visitComparisonExpression(EsqlBaseParser.ComparisonExpressionContext ctx);
/**
* Visit a parse tree produced by the {@code operatorExpressionDefault}
* labeled alternative in {@link EsqlBaseParser#operatorExpression}.
@@ -750,15 +756,17 @@ public interface EsqlBaseParserVisitor extends ParseTreeVisitor {
*/
T visitJoinTarget(EsqlBaseParser.JoinTargetContext ctx);
/**
- * Visit a parse tree produced by {@link EsqlBaseParser#joinCondition}.
+ * Visit a parse tree produced by the {@code fieldBasedLookupJoin}
+ * labeled alternative in {@link EsqlBaseParser#joinCondition}.
* @param ctx the parse tree
* @return the visitor result
*/
- T visitJoinCondition(EsqlBaseParser.JoinConditionContext ctx);
+ T visitFieldBasedLookupJoin(EsqlBaseParser.FieldBasedLookupJoinContext ctx);
/**
- * Visit a parse tree produced by {@link EsqlBaseParser#joinPredicate}.
+ * Visit a parse tree produced by the {@code expressionBasedLookupJoin}
+ * labeled alternative in {@link EsqlBaseParser#joinCondition}.
* @param ctx the parse tree
* @return the visitor result
*/
- T visitJoinPredicate(EsqlBaseParser.JoinPredicateContext ctx);
+ T visitExpressionBasedLookupJoin(EsqlBaseParser.ExpressionBasedLookupJoinContext ctx);
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java
index 451be4db743da..d3ece791c3047 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ExpressionBuilder.java
@@ -608,6 +608,11 @@ public Expression visitArithmeticBinary(EsqlBaseParser.ArithmeticBinaryContext c
@Override
public Expression visitComparison(EsqlBaseParser.ComparisonContext ctx) {
+ return expression(ctx.comparisonExpression());
+ }
+
+ @Override
+ public Expression visitComparisonExpression(EsqlBaseParser.ComparisonExpressionContext ctx) {
Expression left = expression(ctx.left);
Expression right = expression(ctx.right);
TerminalNode op = (TerminalNode) ctx.comparisonOperator().getChild(0);
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java
index 974f5fa485e47..b874a02800a71 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java
@@ -46,6 +46,10 @@
import org.elasticsearch.xpack.esql.expression.function.UnresolvedFunction;
import org.elasticsearch.xpack.esql.expression.function.aggregate.Sum;
import org.elasticsearch.xpack.esql.expression.function.aggregate.SummationMode;
+import org.elasticsearch.xpack.esql.expression.predicate.Predicates;
+import org.elasticsearch.xpack.esql.expression.predicate.logical.Not;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.EsqlBinaryComparison;
import org.elasticsearch.xpack.esql.plan.IndexPattern;
import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
import org.elasticsearch.xpack.esql.plan.logical.ChangePoint;
@@ -593,7 +597,7 @@ public PlanFactory visitLookupCommand(EsqlBaseParser.LookupCommandContext ctx) {
Literal tableName = Literal.keyword(source, visitIndexPattern(List.of(ctx.indexPattern())));
- return p -> new Lookup(source, p, tableName, matchFields, null /* localRelation will be resolved later*/);
+ return p -> new Lookup(source, p, tableName, matchFields, null /* localRelation will be resolved later*/, null);
}
@Override
@@ -638,9 +642,29 @@ public PlanFactory visitJoinCommand(EsqlBaseParser.JoinCommandContext ctx) {
);
var condition = ctx.joinCondition();
+ var joinInfo = typedParsing(this, condition, JoinInfo.class);
- // ON only with un-qualified names for now
- var predicates = expressions(condition.joinPredicate());
+ return p -> {
+ boolean hasRemotes = p.anyMatch(node -> {
+ if (node instanceof UnresolvedRelation r) {
+ return Arrays.stream(Strings.splitStringByCommaToArray(r.indexPattern().indexPattern()))
+ .anyMatch(RemoteClusterAware::isRemoteIndexName);
+ } else {
+ return false;
+ }
+ });
+ if (hasRemotes && EsqlCapabilities.Cap.ENABLE_LOOKUP_JOIN_ON_REMOTE.isEnabled() == false) {
+ throw new ParsingException(source, "remote clusters are not supported with LOOKUP JOIN");
+ }
+ return new LookupJoin(source, p, right, joinInfo.joinFields(), hasRemotes, Predicates.combineAnd(joinInfo.joinExpressions()));
+ };
+ }
+
+ private record JoinInfo(List joinFields, List joinExpressions) {}
+
+ @Override
+ public JoinInfo visitFieldBasedLookupJoin(EsqlBaseParser.FieldBasedLookupJoinContext ctx) {
+ var predicates = visitList(this, ctx.qualifiedName(), Expression.class);
List joinFields = new ArrayList<>(predicates.size());
for (var f : predicates) {
// verify each field is an unresolved attribute
@@ -654,12 +678,43 @@ public PlanFactory visitJoinCommand(EsqlBaseParser.JoinCommandContext ctx) {
}
joinFields.add(ua);
} else {
- throw new ParsingException(f.source(), "JOIN ON clause only supports fields at the moment, found [{}]", f.sourceText());
+ throw new ParsingException(
+ f.source(),
+ "JOIN ON clause only supports fields or AND of Binary Expressions at the moment, found [{}]",
+ f.sourceText()
+ );
+ }
+ }
+ validateJoinFields(joinFields);
+ return new JoinInfo(joinFields, emptyList());
+ }
+
+ @Override
+ public JoinInfo visitExpressionBasedLookupJoin(EsqlBaseParser.ExpressionBasedLookupJoinContext ctx) {
+ var predicates = visitList(this, ctx.comparisonExpression(), Expression.class);
+ List joinFields = new ArrayList<>(predicates.size());
+ List joinExpressions = new ArrayList<>(predicates.size());
+ for (var f : predicates) {
+ f = handleNegationOfEquals(f);
+ if (f instanceof EsqlBinaryComparison comparison
+ && comparison.left() instanceof UnresolvedAttribute left
+ && comparison.right() instanceof UnresolvedAttribute) {
+ joinFields.add(left);
+ joinExpressions.add(f);
+ } else {
+ throw new ParsingException(
+ f.source(),
+ "JOIN ON clause only supports fields or AND of Binary Expressions at the moment, found [{}]",
+ f.sourceText()
+ );
}
}
+ validateJoinFields(joinFields);
+ return new JoinInfo(joinFields, joinExpressions);
+ }
- var matchFieldsCount = joinFields.size();
- if (matchFieldsCount > 1) {
+ private void validateJoinFields(List joinFields) {
+ if (joinFields.size() > 1) {
Set matchFieldNames = new LinkedHashSet<>();
for (Attribute field : joinFields) {
if (matchFieldNames.add(field.name()) == false) {
@@ -669,24 +724,16 @@ public PlanFactory visitJoinCommand(EsqlBaseParser.JoinCommandContext ctx) {
field.name()
);
}
-
}
}
+ }
- return p -> {
- boolean hasRemotes = p.anyMatch(node -> {
- if (node instanceof UnresolvedRelation r) {
- return Arrays.stream(Strings.splitStringByCommaToArray(r.indexPattern().indexPattern()))
- .anyMatch(RemoteClusterAware::isRemoteIndexName);
- } else {
- return false;
- }
- });
- if (hasRemotes && EsqlCapabilities.Cap.ENABLE_LOOKUP_JOIN_ON_REMOTE.isEnabled() == false) {
- throw new ParsingException(source, "remote clusters are not supported with LOOKUP JOIN");
- }
- return new LookupJoin(source, p, right, joinFields, hasRemotes);
- };
+ private Expression handleNegationOfEquals(Expression f) {
+ if (f instanceof Not not && not.children().size() == 1 && not.children().get(0) instanceof Equals equals) {
+ // we only support NOT on Equals, by converting it to NotEquals
+ return equals.negate();
+ }
+ return f;
}
private void checkForRemoteClusters(LogicalPlan plan, Source source, String commandName) {
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/InlineStats.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/InlineStats.java
index be34631ec8149..4531e8958915f 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/InlineStats.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/InlineStats.java
@@ -118,7 +118,7 @@ private JoinConfig joinConfig() {
}
}
}
- return new JoinConfig(JoinTypes.LEFT, namedGroupings, leftFields, rightFields);
+ return new JoinConfig(JoinTypes.LEFT, namedGroupings, leftFields, rightFields, null);
}
@Override
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Lookup.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Lookup.java
index 56dae7b1f16c0..ce1d875e98e15 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Lookup.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/Lookup.java
@@ -7,6 +7,7 @@
package org.elasticsearch.xpack.esql.plan.logical;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@@ -44,18 +45,21 @@ public class Lookup extends UnaryPlan implements SurrogateLogicalPlan, Telemetry
// afterward, it is converted into a Join (BinaryPlan) hence why here it is not a child
private final LocalRelation localRelation;
private List lazyOutput;
+ private final Expression joinOnConditions;
public Lookup(
Source source,
LogicalPlan child,
Expression tableName,
List matchFields,
- @Nullable LocalRelation localRelation
+ @Nullable LocalRelation localRelation,
+ @Nullable Expression joinOnConditions
) {
super(source, child);
this.tableName = tableName;
this.matchFields = matchFields;
this.localRelation = localRelation;
+ this.joinOnConditions = joinOnConditions;
}
public Lookup(StreamInput in) throws IOException {
@@ -63,6 +67,11 @@ public Lookup(StreamInput in) throws IOException {
this.tableName = in.readNamedWriteable(Expression.class);
this.matchFields = in.readNamedWriteableCollectionAsList(Attribute.class);
this.localRelation = in.readBoolean() ? new LocalRelation(in) : null;
+ if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ this.joinOnConditions = in.readOptionalNamedWriteable(Expression.class);
+ } else {
+ this.joinOnConditions = null;
+ }
}
@Override
@@ -77,6 +86,11 @@ public void writeTo(StreamOutput out) throws IOException {
out.writeBoolean(true);
localRelation.writeTo(out);
}
+ if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ out.writeOptionalNamedWriteable(joinOnConditions);
+ } else if (joinOnConditions != null) {
+ throw new IOException("LOOKUP with ON conditions is not supported on remote node");
+ }
}
@Override
@@ -115,7 +129,11 @@ public JoinConfig joinConfig() {
}
}
}
- return new JoinConfig(JoinTypes.LEFT, matchFields, leftFields, rightFields);
+ return new JoinConfig(JoinTypes.LEFT, matchFields, leftFields, rightFields, joinOnConditions);
+ }
+
+ public Expression getJoinOnConditions() {
+ return joinOnConditions;
}
@Override
@@ -125,12 +143,12 @@ public boolean expressionsResolved() {
@Override
public UnaryPlan replaceChild(LogicalPlan newChild) {
- return new Lookup(source(), newChild, tableName, matchFields, localRelation);
+ return new Lookup(source(), newChild, tableName, matchFields, localRelation, joinOnConditions);
}
@Override
protected NodeInfo extends LogicalPlan> info() {
- return NodeInfo.create(this, Lookup::new, child(), tableName, matchFields, localRelation);
+ return NodeInfo.create(this, Lookup::new, child(), tableName, matchFields, localRelation, joinOnConditions);
}
@Override
@@ -158,11 +176,12 @@ public boolean equals(Object o) {
Lookup lookup = (Lookup) o;
return Objects.equals(tableName, lookup.tableName)
&& Objects.equals(matchFields, lookup.matchFields)
- && Objects.equals(localRelation, lookup.localRelation);
+ && Objects.equals(localRelation, lookup.localRelation)
+ && Objects.equals(joinOnConditions, lookup.joinOnConditions);
}
@Override
public int hashCode() {
- return Objects.hash(super.hashCode(), tableName, matchFields, localRelation);
+ return Objects.hash(super.hashCode(), tableName, matchFields, localRelation, joinOnConditions);
}
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java
index 1c3f341321f6f..304ef551ec6c3 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/InlineJoin.java
@@ -163,7 +163,7 @@ public InlineJoin(
List leftFields,
List rightFields
) {
- super(source, left, right, type, matchFields, leftFields, rightFields);
+ super(source, left, right, type, matchFields, leftFields, rightFields, null);
}
private static InlineJoin readFrom(StreamInput in) throws IOException {
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java
index 66796c6ecb02d..5d8329ef3049f 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java
@@ -7,6 +7,7 @@
package org.elasticsearch.xpack.esql.plan.logical.join;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@@ -15,6 +16,7 @@
import org.elasticsearch.xpack.esql.common.Failures;
import org.elasticsearch.xpack.esql.core.expression.Attribute;
import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.expression.Expressions;
import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
@@ -23,6 +25,7 @@
import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput;
import org.elasticsearch.xpack.esql.plan.logical.BinaryPlan;
import org.elasticsearch.xpack.esql.plan.logical.ExecutesOn;
+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.PipelineBreaker;
@@ -109,10 +112,11 @@ public Join(
JoinType type,
List matchFields,
List leftFields,
- List rightFields
+ List rightFields,
+ Expression joinOnConditions
) {
super(source, left, right);
- this.config = new JoinConfig(type, matchFields, leftFields, rightFields);
+ this.config = new JoinConfig(type, matchFields, leftFields, rightFields, joinOnConditions);
}
public Join(StreamInput in) throws IOException {
@@ -124,7 +128,16 @@ public Join(StreamInput in) throws IOException {
public void writeTo(StreamOutput out) throws IOException {
source().writeTo(out);
out.writeNamedWriteable(left());
- out.writeNamedWriteable(right());
+ LogicalPlan rightToSerialize = right();
+ if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_PRE_JOIN_FILTER) == false) {
+ // Prior to TransportVersions.ESQL_LOOKUP_JOIN_PRE_JOIN_FILTER
+ // we do not support a filter on top of the right side of the join
+ // As we consider the filters optional, we remove them here
+ while (rightToSerialize instanceof Filter filter) {
+ rightToSerialize = filter.child();
+ }
+ }
+ out.writeNamedWriteable(rightToSerialize);
config.writeTo(out);
}
@@ -149,7 +162,8 @@ protected NodeInfo info() {
config.type(),
config.matchFields(),
config.leftFields(),
- config.rightFields()
+ config.rightFields(),
+ config.joinOnConditions()
);
}
@@ -200,10 +214,33 @@ public static List computeOutput(List leftOutput, List output;
// TODO: make the other side nullable
if (LEFT.equals(joinType)) {
- // right side becomes nullable and overrides left except for join keys, which we preserve from the left
- AttributeSet rightKeys = AttributeSet.of(config.rightFields());
- List rightOutputWithoutMatchFields = rightOutput.stream().filter(attr -> rightKeys.contains(attr) == false).toList();
- output = mergeOutputAttributes(rightOutputWithoutMatchFields, leftOutput);
+ if (config.joinOnConditions() == null) {
+ // right side becomes nullable and overrides left except for join keys, which we preserve from the left
+ AttributeSet rightKeys = AttributeSet.of(config.rightFields());
+ List rightOutputWithoutMatchFields = rightOutput.stream()
+ .filter(attr -> rightKeys.contains(attr) == false)
+ .toList();
+ output = mergeOutputAttributes(rightOutputWithoutMatchFields, leftOutput);
+ } else {
+ // suppose we have
+ // index1 a, b, c, d
+ // index2 c, d, e, f
+ // we have expression join on (index1.b > index2.c AND index1.d > index2.d)
+ // We keep a,b,d from the left side (a, b only on left, d is join key, so we keep it from the left)
+ // we keep c,e,f from the right side(c is not part of the join key on the left so gets shadowed, e,f only on right)
+ Set leftJoinKeyNames = config.leftFields()
+ .stream()
+ .map(Attribute::name)
+ .collect(java.util.stream.Collectors.toSet());
+ // as we can do (left_key > right_key) now as join condition, we want to preserve the right_key
+ // unless it is also a key on the left side,
+ // but we need to do name equality only
+ List rightOutputWithoutMatchFields = rightOutput.stream()
+ .filter(attr -> leftJoinKeyNames.contains(attr.name()) == false)
+ .toList();
+ output = mergeOutputAttributes(rightOutputWithoutMatchFields, leftOutput);
+ }
+
} else {
throw new IllegalArgumentException(joinType.joinName() + " unsupported");
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/JoinConfig.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/JoinConfig.java
index 383606d6ccbed..2af78552f46ce 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/JoinConfig.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/JoinConfig.java
@@ -7,49 +7,145 @@
package org.elasticsearch.xpack.esql.plan.logical.join;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
+import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;
import org.elasticsearch.xpack.esql.core.capabilities.Resolvables;
import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import java.io.IOException;
import java.util.List;
+import java.util.Objects;
/**
* Configuration for a {@code JOIN} style operation.
- * @param matchFields fields either from the left or right fields which decide which side is kept
- * @param leftFields matched with the right fields
- * @param rightFields matched with the left fields
*/
// TODO: this class needs refactoring into a more general form (expressions) since it's currently contains
// both the condition (equi-join) between the left and right field as well as the output of the join keys
// which makes sense only for USING clause - which is better resolved in the analyzer (based on the names)
// hence why for now the attributes are set inside the analyzer
-public record JoinConfig(JoinType type, List matchFields, List leftFields, List rightFields)
- implements
- Writeable {
+public final class JoinConfig implements Writeable {
+ private final JoinType type;
+ private final List matchFields;
+ private final List leftFields;
+ private final List rightFields;
+ private final Expression joinOnConditions;
+
+ /**
+ * @param matchFields fields either from the left or right fields which decide which side is kept
+ * @param leftFields matched with the right fields
+ * @param rightFields matched with the left fields
+ */
+ public JoinConfig(
+ JoinType type,
+ List matchFields,
+ List leftFields,
+ List rightFields,
+ Expression joinOnConditions
+ ) {
+ this.type = type;
+ this.matchFields = matchFields;
+ this.leftFields = leftFields;
+ this.rightFields = rightFields;
+ this.joinOnConditions = joinOnConditions;
+ }
+
public JoinConfig(StreamInput in) throws IOException {
this(
JoinTypes.readFrom(in),
in.readNamedWriteableCollectionAsList(Attribute.class),
in.readNamedWriteableCollectionAsList(Attribute.class),
- in.readNamedWriteableCollectionAsList(Attribute.class)
+ in.readNamedWriteableCollectionAsList(Attribute.class),
+ readJoinConditions(in)
);
}
+ private static Expression readJoinConditions(StreamInput in) throws IOException {
+ if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ return in.readOptionalNamedWriteable(Expression.class);
+ }
+ return null;
+ }
+
@Override
public void writeTo(StreamOutput out) throws IOException {
type.writeTo(out);
out.writeNamedWriteableCollection(matchFields);
out.writeNamedWriteableCollection(leftFields);
out.writeNamedWriteableCollection(rightFields);
+ if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ out.writeOptionalNamedWriteable(joinOnConditions);
+ } else if (joinOnConditions != null) {
+ throw new EsqlIllegalArgumentException("LOOKUP JOIN with ON conditions is not supported on remote node");
+ }
}
public boolean expressionsResolved() {
return type.resolved()
&& Resolvables.resolved(matchFields)
&& Resolvables.resolved(leftFields)
- && Resolvables.resolved(rightFields);
+ && Resolvables.resolved(rightFields)
+ && (joinOnConditions == null || joinOnConditions.resolved());
+ }
+
+ public JoinType type() {
+ return type;
+ }
+
+ public List matchFields() {
+ return matchFields;
+ }
+
+ public List leftFields() {
+ return leftFields;
+ }
+
+ public List rightFields() {
+ return rightFields;
+ }
+
+ public Expression joinOnConditions() {
+ return joinOnConditions;
}
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == this) return true;
+ if (obj == null || obj.getClass() != this.getClass()) return false;
+ var that = (JoinConfig) obj;
+ return Objects.equals(this.type, that.type)
+ && Objects.equals(this.matchFields, that.matchFields)
+ && Objects.equals(this.leftFields, that.leftFields)
+ && Objects.equals(this.rightFields, that.rightFields)
+ && Objects.equals(this.joinOnConditions, that.joinOnConditions);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(type, matchFields, leftFields, rightFields, joinOnConditions);
+ }
+
+ @Override
+ public String toString() {
+ return "JoinConfig["
+ + "type="
+ + type
+ + ", "
+ + "matchFields="
+ + matchFields
+ + ", "
+ + "leftFields="
+ + leftFields
+ + ", "
+ + "rightFields="
+ + rightFields
+ + ", "
+ + "joinOnConditions="
+ + joinOnConditions
+ + ']';
+ }
+
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/LookupJoin.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/LookupJoin.java
index 20913e0e27ce7..29ff17e218845 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/LookupJoin.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/LookupJoin.java
@@ -7,10 +7,12 @@
package org.elasticsearch.xpack.esql.plan.logical.join;
+import org.elasticsearch.core.Nullable;
import org.elasticsearch.xpack.esql.capabilities.PostAnalysisVerificationAware;
import org.elasticsearch.xpack.esql.capabilities.TelemetryAware;
import org.elasticsearch.xpack.esql.common.Failures;
import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
import org.elasticsearch.xpack.esql.core.tree.Source;
import org.elasticsearch.xpack.esql.plan.logical.Limit;
@@ -29,12 +31,19 @@
*/
public class LookupJoin extends Join implements SurrogateLogicalPlan, TelemetryAware, PostAnalysisVerificationAware {
- public LookupJoin(Source source, LogicalPlan left, LogicalPlan right, List joinFields, boolean isRemote) {
- this(source, left, right, new UsingJoinType(LEFT, joinFields), emptyList(), emptyList(), emptyList(), isRemote);
+ public LookupJoin(
+ Source source,
+ LogicalPlan left,
+ LogicalPlan right,
+ List joinFields,
+ boolean isRemote,
+ @Nullable Expression joinOnConditions
+ ) {
+ this(source, left, right, new UsingJoinType(LEFT, joinFields), emptyList(), emptyList(), emptyList(), isRemote, joinOnConditions);
}
public LookupJoin(Source source, LogicalPlan left, LogicalPlan right, List joinFields) {
- this(source, left, right, new UsingJoinType(LEFT, joinFields), emptyList(), emptyList(), emptyList(), false);
+ this(source, left, right, new UsingJoinType(LEFT, joinFields), emptyList(), emptyList(), emptyList(), false, null);
}
public LookupJoin(
@@ -45,9 +54,10 @@ public LookupJoin(
List joinFields,
List leftFields,
List rightFields,
- boolean isRemote
+ boolean isRemote,
+ Expression joinOnConditions
) {
- this(source, left, right, new JoinConfig(type, joinFields, leftFields, rightFields), isRemote);
+ this(source, left, right, new JoinConfig(type, joinFields, leftFields, rightFields, joinOnConditions), isRemote);
}
public LookupJoin(Source source, LogicalPlan left, LogicalPlan right, JoinConfig joinConfig) {
@@ -83,7 +93,8 @@ protected NodeInfo info() {
config().matchFields(),
config().leftFields(),
config().rightFields(),
- isRemote()
+ isRemote(),
+ config().joinOnConditions()
);
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/BinaryExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/BinaryExec.java
index 9a1b76205b595..dfb8b30bb44aa 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/BinaryExec.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/BinaryExec.java
@@ -7,6 +7,7 @@
package org.elasticsearch.xpack.esql.plan.physical;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
import org.elasticsearch.xpack.esql.core.tree.Source;
@@ -49,7 +50,17 @@ public PhysicalPlan right() {
public void writeTo(StreamOutput out) throws IOException {
Source.EMPTY.writeTo(out);
out.writeNamedWriteable(left);
- out.writeNamedWriteable(right);
+ PhysicalPlan rightToSerialize = right;
+ if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_PRE_JOIN_FILTER) == false
+ && this instanceof LookupJoinExec) {
+ // Prior to TransportVersions.ESQL_LOOKUP_JOIN_PRE_JOIN_FILTER
+ // we do not support a filter on top of the right side of the join
+ // As we consider the filters optional, we remove them here
+ while (rightToSerialize instanceof FilterExec filterExec) {
+ rightToSerialize = filterExec.child();
+ }
+ }
+ out.writeNamedWriteable(rightToSerialize);
}
@Override
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java
index 2aff38993aa98..ff8adbaf536e2 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java
@@ -7,11 +7,14 @@
package org.elasticsearch.xpack.esql.plan.physical;
+import org.elasticsearch.TransportVersions;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException;
import org.elasticsearch.xpack.esql.core.expression.Attribute;
import org.elasticsearch.xpack.esql.core.expression.AttributeSet;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.expression.Expressions;
import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
import org.elasticsearch.xpack.esql.core.tree.Source;
@@ -37,6 +40,7 @@ public class LookupJoinExec extends BinaryExec implements EstimatesRowSize {
* the right hand side by a {@link EsQueryExec}, and thus lose the information of which fields we'll get from the lookup index.
*/
private final List addedFields;
+ private final Expression joinOnConditions;
private List lazyOutput;
public LookupJoinExec(
@@ -45,12 +49,14 @@ public LookupJoinExec(
PhysicalPlan lookup,
List leftFields,
List rightFields,
- List addedFields
+ List addedFields,
+ Expression joinOnConditions
) {
super(source, left, lookup);
this.leftFields = leftFields;
this.rightFields = rightFields;
this.addedFields = addedFields;
+ this.joinOnConditions = joinOnConditions;
}
private LookupJoinExec(StreamInput in) throws IOException {
@@ -58,6 +64,11 @@ private LookupJoinExec(StreamInput in) throws IOException {
this.leftFields = in.readNamedWriteableCollectionAsList(Attribute.class);
this.rightFields = in.readNamedWriteableCollectionAsList(Attribute.class);
this.addedFields = in.readNamedWriteableCollectionAsList(Attribute.class);
+ if (in.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ this.joinOnConditions = in.readOptionalNamedWriteable(Expression.class);
+ } else {
+ this.joinOnConditions = null;
+ }
}
@Override
@@ -66,6 +77,11 @@ public void writeTo(StreamOutput out) throws IOException {
out.writeNamedWriteableCollection(leftFields);
out.writeNamedWriteableCollection(rightFields);
out.writeNamedWriteableCollection(addedFields);
+ if (out.getTransportVersion().onOrAfter(TransportVersions.ESQL_LOOKUP_JOIN_ON_EXPRESSION)) {
+ out.writeOptionalNamedWriteable(joinOnConditions);
+ } else if (joinOnConditions != null) {
+ throw new EsqlIllegalArgumentException("LOOKUP JOIN with ON conditions is not supported on remote node");
+ }
}
@Override
@@ -136,12 +152,12 @@ public AttributeSet rightReferences() {
@Override
public LookupJoinExec replaceChildren(PhysicalPlan left, PhysicalPlan right) {
- return new LookupJoinExec(source(), left, right, leftFields, rightFields, addedFields);
+ return new LookupJoinExec(source(), left, right, leftFields, rightFields, addedFields, joinOnConditions);
}
@Override
protected NodeInfo extends PhysicalPlan> info() {
- return NodeInfo.create(this, LookupJoinExec::new, left(), right(), leftFields, rightFields, addedFields);
+ return NodeInfo.create(this, LookupJoinExec::new, left(), right(), leftFields, rightFields, addedFields, joinOnConditions);
}
@Override
@@ -156,11 +172,22 @@ public boolean equals(Object o) {
return false;
}
LookupJoinExec other = (LookupJoinExec) o;
- return leftFields.equals(other.leftFields) && rightFields.equals(other.rightFields) && addedFields.equals(other.addedFields);
+ return leftFields.equals(other.leftFields)
+ && rightFields.equals(other.rightFields)
+ && addedFields.equals(other.addedFields)
+ && Objects.equals(joinOnConditions, other.joinOnConditions);
}
@Override
public int hashCode() {
- return Objects.hash(super.hashCode(), leftFields, rightFields, addedFields);
+ return Objects.hash(super.hashCode(), leftFields, rightFields, addedFields, joinOnConditions);
+ }
+
+ public boolean isOnJoinExpression() {
+ return joinOnConditions != null;
+ }
+
+ public Expression joinOnConditions() {
+ return joinOnConditions;
}
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
index 5ae15f4c0e844..f8f554b9d88e9 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java
@@ -92,7 +92,9 @@
import org.elasticsearch.xpack.esql.inference.XContentRowEncoder;
import org.elasticsearch.xpack.esql.inference.completion.CompletionOperator;
import org.elasticsearch.xpack.esql.inference.rerank.RerankOperator;
+import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
import org.elasticsearch.xpack.esql.plan.logical.Fork;
+import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
import org.elasticsearch.xpack.esql.plan.physical.ChangePointExec;
import org.elasticsearch.xpack.esql.plan.physical.DissectExec;
@@ -105,6 +107,7 @@
import org.elasticsearch.xpack.esql.plan.physical.ExchangeSourceExec;
import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
import org.elasticsearch.xpack.esql.plan.physical.GrokExec;
import org.elasticsearch.xpack.esql.plan.physical.HashJoinExec;
import org.elasticsearch.xpack.esql.plan.physical.LimitExec;
@@ -740,8 +743,8 @@ private PhysicalOperation planLookupJoin(LookupJoinExec join, LocalExecutionPlan
}
Layout layout = layoutBuilder.build();
- EsQueryExec localSourceExec = (EsQueryExec) join.lookup();
- if (localSourceExec.indexMode() != IndexMode.LOOKUP) {
+ EsRelation esRelation = findEsRelation(join.lookup());
+ if (esRelation == null || esRelation.indexMode() != IndexMode.LOOKUP) {
throw new IllegalArgumentException("can't plan [" + join + "]");
}
@@ -749,10 +752,10 @@ private PhysicalOperation planLookupJoin(LookupJoinExec join, LocalExecutionPlan
// 1. We've just got one entry - this should be the one relevant to the join, and it should be for this cluster
// 2. We have got multiple entries - this means each cluster has its own one, and we should extract one relevant for this cluster
Map.Entry entry;
- if (localSourceExec.indexNameWithModes().size() == 1) {
- entry = localSourceExec.indexNameWithModes().entrySet().iterator().next();
+ if (esRelation.indexNameWithModes().size() == 1) {
+ entry = esRelation.indexNameWithModes().entrySet().iterator().next();
} else {
- var maybeEntry = localSourceExec.indexNameWithModes()
+ var maybeEntry = esRelation.indexNameWithModes()
.entrySet()
.stream()
.filter(e -> RemoteClusterAware.parseClusterAlias(e.getKey()).equals(clusterAlias))
@@ -786,9 +789,23 @@ private PhysicalOperation planLookupJoin(LookupJoinExec join, LocalExecutionPlan
if (input == null) {
throw new IllegalArgumentException("can't plan [" + join + "][" + left + "]");
}
- matchFields.add(new MatchConfig(right, input));
+ // we support 2 types of joins: Field name joins and Expression joins
+ // for Field name join, we do not ship any join on expression.
+ // we built the Lucene query on the field name that is passed in the MatchConfig.fieldName
+ // so for Field name we need to pass the attribute name from the right side, because that is needed to build the query
+ // For expression joins, we pass an expression such as left_id > right_id.
+ // So in this case we pass in left_id as the field name, because that is what we are shipping to the lookup node
+ // The lookup node will replace that name, with the actual values for each row and perform the lookup join
+ // We need to pass the left name, because we need to know what data we have shipped.
+ // It is not acceptable to just use the left or right side of the operator because the same field can be joined multiple times
+ // e.g. LOOKUP JOIN ON left_id < right_id_1 and left_id >= right_id_2
+ // we want to be able to optimize this in the future and only ship the left_id once
+ FieldAttribute.FieldName fieldName = right.exactAttribute().fieldName();
+ if (join.isOnJoinExpression()) {
+ fieldName = new FieldAttribute.FieldName(left.name());
+ }
+ matchFields.add(new MatchConfig(fieldName, input));
}
-
return source.with(
new LookupFromIndexOperator.Factory(
matchFields,
@@ -796,15 +813,27 @@ private PhysicalOperation planLookupJoin(LookupJoinExec join, LocalExecutionPlan
parentTask,
context.queryPragmas().enrichMaxWorkers(),
ctx -> lookupFromIndexService,
- localSourceExec.indexPattern(),
+ esRelation.indexPattern(),
indexName,
join.addedFields().stream().map(f -> (NamedExpression) f).toList(),
- join.source()
+ join.source(),
+ join.right(),
+ join.joinOnConditions()
),
layout
);
}
+ private static EsRelation findEsRelation(PhysicalPlan node) {
+ if (node instanceof FragmentExec fragmentExec) {
+ List esRelations = fragmentExec.fragment().collectFirstChildren(x -> x instanceof EsRelation);
+ if (esRelations.size() == 1) {
+ return (EsRelation) esRelations.get(0);
+ }
+ }
+ return null;
+ }
+
private PhysicalOperation planLocal(LocalSourceExec localSourceExec, LocalExecutionPlannerContext context) {
Layout.Builder layout = new Layout.Builder();
layout.append(localSourceExec.output());
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java
index f956bdf34089f..4e2f1f9c1de77 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/PlannerUtils.java
@@ -49,6 +49,7 @@
import org.elasticsearch.xpack.esql.plan.physical.ExchangeSinkExec;
import org.elasticsearch.xpack.esql.plan.physical.ExchangeSourceExec;
import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
+import org.elasticsearch.xpack.esql.plan.physical.LookupJoinExec;
import org.elasticsearch.xpack.esql.plan.physical.MergeExec;
import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan;
import org.elasticsearch.xpack.esql.planner.mapper.LocalMapper;
@@ -208,8 +209,18 @@ public static PhysicalPlan localPlan(
) {
final LocalMapper localMapper = new LocalMapper();
var isCoordPlan = new Holder<>(Boolean.TRUE);
+ List lookupJoinExecRightChildren = plan.collect(LookupJoinExec.class::isInstance)
+ .stream()
+ .map(x -> ((LookupJoinExec) x).right())
+ .toList();
var localPhysicalPlan = plan.transformUp(FragmentExec.class, f -> {
+ if (lookupJoinExecRightChildren.contains(f)) {
+ // Do not optimize the right child of a lookup join exec
+ // The data node does not have the right stats to perform the optimization because the stats are on the lookup node
+ // Also we only ship logical plans across the network, so the plan needs to remain logical
+ return f;
+ }
isCoordPlan.set(Boolean.FALSE);
var optimizedFragment = logicalOptimizer.localOptimize(f.fragment());
var physicalFragment = localMapper.map(optimizedFragment);
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java
index a1671bffc5c25..05758721db503 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java
@@ -23,6 +23,8 @@
import org.elasticsearch.xpack.esql.plan.logical.join.JoinConfig;
import org.elasticsearch.xpack.esql.plan.logical.join.JoinTypes;
import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec;
+import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
import org.elasticsearch.xpack.esql.plan.physical.HashJoinExec;
import org.elasticsearch.xpack.esql.plan.physical.LimitExec;
import org.elasticsearch.xpack.esql.plan.physical.LocalSourceExec;
@@ -111,11 +113,42 @@ private PhysicalPlan mapBinary(BinaryPlan binary) {
join.rightOutputFields()
);
}
- if (right instanceof EsSourceExec source && source.indexMode() == IndexMode.LOOKUP) {
- return new LookupJoinExec(join.source(), left, right, config.leftFields(), config.rightFields(), join.rightOutputFields());
+ LookupJoinExec lookupJoinExec = getLookupJoinExec(join, right, left, config);
+ if (lookupJoinExec == null && right instanceof FilterExec filterExec) {
+ lookupJoinExec = getLookupJoinExec(join, filterExec.child(), left, config);
}
+ if (lookupJoinExec != null) {
+ // we want to do local physical planning on the lookup node eventually for the right side of the lookup join
+ // so here we will wrap the logical plan with a FragmentExec and keep it as is
+ FragmentExec fragmentExec = new FragmentExec(binary.right());
+ return new LookupJoinExec(
+ join.source(),
+ left,
+ fragmentExec,
+ config.leftFields(),
+ config.rightFields(),
+ join.rightOutputFields(),
+ config.joinOnConditions()
+ );
+ }
+ return MapperUtils.unsupported(binary);
}
return MapperUtils.unsupported(binary);
}
+
+ private static LookupJoinExec getLookupJoinExec(Join join, PhysicalPlan right, PhysicalPlan left, JoinConfig config) {
+ if (right instanceof EsSourceExec source && source.indexMode() == IndexMode.LOOKUP) {
+ return new LookupJoinExec(
+ join.source(),
+ left,
+ right,
+ config.leftFields(),
+ config.rightFields(),
+ join.rightOutputFields(),
+ config.joinOnConditions()
+ );
+ }
+ return null;
+ }
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java
index aac1d58e5f7f1..a89f9acab20e1 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java
@@ -16,6 +16,7 @@
import org.elasticsearch.xpack.esql.plan.logical.BinaryPlan;
import org.elasticsearch.xpack.esql.plan.logical.Enrich;
import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
+import org.elasticsearch.xpack.esql.plan.logical.Filter;
import org.elasticsearch.xpack.esql.plan.logical.Fork;
import org.elasticsearch.xpack.esql.plan.logical.LeafPlan;
import org.elasticsearch.xpack.esql.plan.logical.Limit;
@@ -226,16 +227,36 @@ private PhysicalPlan mapBinary(BinaryPlan bp) {
join.rightOutputFields()
);
}
- if (right instanceof FragmentExec fragment
- && fragment.fragment() instanceof EsRelation relation
- && relation.indexMode() == IndexMode.LOOKUP) {
- return new LookupJoinExec(join.source(), left, right, config.leftFields(), config.rightFields(), join.rightOutputFields());
+ if (right instanceof FragmentExec fragment) {
+ boolean isIndexModeLookup = isIndexModeLookup(fragment);
+ if (isIndexModeLookup) {
+ return new LookupJoinExec(
+ join.source(),
+ left,
+ right,
+ config.leftFields(),
+ config.rightFields(),
+ join.rightOutputFields(),
+ config.joinOnConditions()
+ );
+ }
}
}
-
return MapperUtils.unsupported(bp);
}
+ private static boolean isIndexModeLookup(FragmentExec fragment) {
+ // we support 2 cases:
+ // EsRelation in index_mode=lookup
+ boolean isIndexModeLookup = fragment.fragment() instanceof EsRelation relation && relation.indexMode() == IndexMode.LOOKUP;
+ // or Filter(EsRelation) in index_mode=lookup
+ isIndexModeLookup = isIndexModeLookup
+ || fragment.fragment() instanceof Filter filter
+ && filter.child() instanceof EsRelation relation
+ && relation.indexMode() == IndexMode.LOOKUP;
+ return isIndexModeLookup;
+ }
+
private PhysicalPlan mapFork(Fork fork) {
return new MergeExec(fork.source(), fork.children().stream().map(child -> map(child)).toList(), fork.output());
}
diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlFlags.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlFlags.java
index a7eacf383fc73..5068a464852ba 100644
--- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlFlags.java
+++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlFlags.java
@@ -11,6 +11,8 @@
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
+import java.util.List;
+
public class EsqlFlags {
public static final Setting ESQL_STRING_LIKE_ON_INDEX = Setting.boolSetting(
"esql.query.string_like_on_index",
@@ -39,7 +41,7 @@ public class EsqlFlags {
Setting.Property.NodeScope,
Setting.Property.Dynamic
);
-
+ public static List> ALL_ESQL_FLAGS_SETTINGS = List.of(ESQL_STRING_LIKE_ON_INDEX, ESQL_ROUNDTO_PUSHDOWN_THRESHOLD);
private final boolean stringLikeOnIndex;
private final int roundToPushdownThreshold;
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java
index 01242b19e3074..90db211a877db 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java
@@ -122,15 +122,15 @@ public void testTooBigQuery() {
public void testJoinOnConstant() {
assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled());
assertEquals(
- "1:55: JOIN ON clause only supports fields at the moment, found [123]",
+ "1:58: mismatched input '' expecting {'::', '==', '!=', '<', '<=', '>', '>=', '+', '-', '*', '/', '%'}",
error("row languages = 1, gender = \"f\" | lookup join test on 123")
);
assertEquals(
- "1:55: JOIN ON clause only supports fields at the moment, found [\"abc\"]",
+ "1:60: mismatched input '' expecting {'::', '==', '!=', '<', '<=', '>', '>=', '+', '-', '*', '/', '%'}",
error("row languages = 1, gender = \"f\" | lookup join test on \"abc\"")
);
assertEquals(
- "1:55: JOIN ON clause only supports fields at the moment, found [false]",
+ "1:60: mismatched input '' expecting {'::', '==', '!=', '<', '<=', '>', '>=', '+', '-', '*', '/', '%'}",
error("row languages = 1, gender = \"f\" | lookup join test on false")
);
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java
index 5f3d22957731a..30fd2c038b38c 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java
@@ -28,9 +28,7 @@
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.compute.data.BlockFactory;
import org.elasticsearch.compute.data.BytesRefBlock;
-import org.elasticsearch.compute.data.BytesRefVector;
import org.elasticsearch.compute.data.IntBlock;
-import org.elasticsearch.compute.data.IntVector;
import org.elasticsearch.compute.data.LongBlock;
import org.elasticsearch.compute.data.LongVector;
import org.elasticsearch.compute.data.Page;
@@ -45,6 +43,7 @@
import org.elasticsearch.core.Releasable;
import org.elasticsearch.core.Releasables;
import org.elasticsearch.core.Tuple;
+import org.elasticsearch.index.IndexMode;
import org.elasticsearch.index.mapper.KeywordFieldMapper;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MapperServiceTestCase;
@@ -62,12 +61,21 @@
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute;
+import org.elasticsearch.xpack.esql.core.tree.Location;
import org.elasticsearch.xpack.esql.core.tree.Source;
import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.core.type.EsField;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan;
+import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
+import org.elasticsearch.xpack.esql.plan.logical.Filter;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
import org.elasticsearch.xpack.esql.planner.EsPhysicalOperationProviders;
+import org.elasticsearch.xpack.esql.plugin.EsqlFlags;
import org.elasticsearch.xpack.esql.plugin.EsqlPlugin;
import org.hamcrest.Matcher;
import org.junit.After;
@@ -76,6 +84,8 @@
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -87,6 +97,7 @@
public class LookupFromIndexOperatorTests extends AsyncOperatorTestCase {
private static final int LOOKUP_SIZE = 1000;
+ private static final int LESS_THAN_VALUE = -40;
private final ThreadPool threadPool = threadPool();
private final Directory lookupIndexDirectory = newDirectory();
private final List releasables = new ArrayList<>();
@@ -135,12 +146,17 @@ protected void assertSimpleOutput(List input, List results) {
for (Page r : results) {
assertThat(r.getBlockCount(), equalTo(numberOfJoinColumns + 2));
LongVector match = r.getBlock(0).asVector();
- BytesRefVector lkwd = r.getBlock(numberOfJoinColumns).asVector();
- IntVector lint = r.getBlock(numberOfJoinColumns + 1).asVector();
+ BytesRefBlock lkwdBlock = r.getBlock(numberOfJoinColumns);
+ IntBlock lintBlock = r.getBlock(numberOfJoinColumns + 1);
for (int p = 0; p < r.getPositionCount(); p++) {
long m = match.getLong(p);
- assertThat(lkwd.getBytesRef(p, new BytesRef()).utf8ToString(), equalTo("l" + m));
- assertThat(lint.getInt(p), equalTo((int) -m));
+ if (m > Math.abs(LESS_THAN_VALUE)) {
+ assertThat(lkwdBlock.getBytesRef(lkwdBlock.getFirstValueIndex(p), new BytesRef()).utf8ToString(), equalTo("l" + m));
+ assertThat(lintBlock.getInt(lintBlock.getFirstValueIndex(p)), equalTo((int) -m));
+ } else {
+ assertTrue("at " + p, lkwdBlock.isNull(p));
+ assertTrue("at " + p, lintBlock.isNull(p));
+ }
}
}
}
@@ -162,6 +178,7 @@ protected Operator.OperatorFactory simple(SimpleOptions options) {
FieldAttribute.FieldName matchField = new FieldAttribute.FieldName("match" + i);
matchFields.add(new MatchConfig(matchField, i, inputDataType));
}
+
return new LookupFromIndexOperator.Factory(
matchFields,
sessionId,
@@ -171,8 +188,26 @@ protected Operator.OperatorFactory simple(SimpleOptions options) {
lookupIndex,
lookupIndex,
loadFields,
- Source.EMPTY
+ Source.EMPTY,
+ buildLessThanFilter(LESS_THAN_VALUE),
+ null
+ );
+ }
+
+ private FragmentExec buildLessThanFilter(int value) {
+ FieldAttribute filterAttribute = new FieldAttribute(
+ Source.EMPTY,
+ "lint",
+ new EsField("lint", DataType.INTEGER, Collections.emptyMap(), true, EsField.TimeSeriesFieldType.NONE)
+ );
+ Expression lessThan = new LessThan(
+ new Source(new Location(0, 0), "lint < " + value),
+ filterAttribute,
+ new Literal(Source.EMPTY, value, DataType.INTEGER)
);
+ EsRelation esRelation = new EsRelation(Source.EMPTY, "test", IndexMode.LOOKUP, Map.of(), List.of());
+ Filter filter = new Filter(Source.EMPTY, esRelation, lessThan);
+ return new FragmentExec(filter);
}
@Override
@@ -180,20 +215,51 @@ protected Matcher expectedDescriptionOfSimple() {
return expectedToStringOfSimple();
}
+ @Override
+ public void testSimpleDescription() {
+ Operator.OperatorFactory factory = simple();
+ String description = factory.describe();
+ assertThat(description, expectedDescriptionOfSimple());
+ try (Operator op = factory.get(driverContext())) {
+ // we use a special pattern here because the description can contain new lines for the right_pre_join_plan
+ String pattern = "^\\w*\\[[\\s\\S]*\\]$";
+ assertThat(description, matchesPattern(pattern));
+ }
+ }
+
@Override
protected Matcher expectedToStringOfSimple() {
StringBuilder sb = new StringBuilder();
- sb.append("LookupOperator\\[index=idx load_fields=\\[lkwd\\{r}#\\d+, lint\\{r}#\\d+]");
+ sb.append("LookupOperator\\[index=idx load_fields=\\[lkwd\\{r}#\\d+, lint\\{r}#\\d+] ");
for (int i = 0; i < numberOfJoinColumns; i++) {
- sb.append(" input_type=LONG match_field=match").append(i).append(" inputChannel=").append(i);
+ sb.append("input_type=LONG match_field=match").append(i).append(" inputChannel=").append(i).append(" ");
}
- sb.append("]");
+ // Accept either the legacy physical plan rendering (FilterExec/EsQueryExec) or the new FragmentExec rendering
+ sb.append("right_pre_join_plan=(?:");
+ // Legacy pattern
+ sb.append("FilterExec\\[lint\\{f}#\\d+ < ")
+ .append(LESS_THAN_VALUE)
+ .append(
+ "\\[INTEGER]]\\n\\\\_EsQueryExec\\[test], indexMode\\[lookup],\\s*(?:query\\[\\]|\\[\\])?,?\\s*"
+ + "limit\\[\\],?\\s*sort\\[(?:\\[\\])?\\]\\s*estimatedRowSize\\[null\\]\\s*queryBuilderAndTags \\[(?:\\[\\]\\])\\]"
+ );
+ sb.append("|");
+ // New FragmentExec pattern - match the actual output format
+ sb.append("FragmentExec\\[filter=null, estimatedRowSize=\\d+, reducer=\\[\\], fragment=\\[<>\\n")
+ .append("Filter\\[lint\\{f}#\\d+ < ")
+ .append(LESS_THAN_VALUE)
+ .append("\\[INTEGER]]\\n")
+ .append("\\\\_EsRelation\\[test]\\[LOOKUP]\\[\\]<>\\]\\]");
+ sb.append(")");
+ sb.append(" join_on_expression=null\\]");
return matchesPattern(sb.toString());
}
private LookupFromIndexService lookupService(DriverContext mainContext) {
boolean beCranky = mainContext.bigArrays().breakerService() instanceof CrankyCircuitBreakerService;
DiscoveryNode localNode = DiscoveryNodeUtils.create("node", "node");
+ var builtInClusterSettings = new HashSet<>(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+ builtInClusterSettings.addAll(EsqlFlags.ALL_ESQL_FLAGS_SETTINGS);
ClusterService clusterService = ClusterServiceUtils.createClusterService(
threadPool,
localNode,
@@ -202,7 +268,7 @@ private LookupFromIndexService lookupService(DriverContext mainContext) {
.put(BlockFactory.LOCAL_BREAKER_OVER_RESERVED_SIZE_SETTING, ByteSizeValue.ofKb(0))
.put(BlockFactory.LOCAL_BREAKER_OVER_RESERVED_MAX_SIZE_SETTING, ByteSizeValue.ofKb(0))
.build(),
- ClusterSettings.createBuiltInClusterSettings()
+ new ClusterSettings(Settings.EMPTY, builtInClusterSettings)
);
IndicesService indicesService = mock(IndicesService.class);
IndexNameExpressionResolver indexNameExpressionResolver = TestIndexNameExpressionResolver.newInstance();
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java
index 9b161388d6cc3..a8f8b1e696ae6 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizerTests.java
@@ -74,6 +74,7 @@
import org.elasticsearch.xpack.esql.optimizer.rules.logical.ExtractAggregateCommonFilter;
import org.elasticsearch.xpack.esql.parser.ParsingException;
import org.elasticsearch.xpack.esql.plan.logical.Enrich;
+import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan;
import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
import org.elasticsearch.xpack.esql.plan.physical.DissectExec;
@@ -85,6 +86,7 @@
import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec;
import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
import org.elasticsearch.xpack.esql.plan.physical.FilterExec;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
import org.elasticsearch.xpack.esql.plan.physical.GrokExec;
import org.elasticsearch.xpack.esql.plan.physical.LimitExec;
import org.elasticsearch.xpack.esql.plan.physical.LocalSourceExec;
@@ -1030,21 +1032,23 @@ public void testMissingFieldsPurgesTheJoinLocallyThroughCommands() {
}
/*
- * LimitExec[1000[INTEGER]]
+ * LimitExec[1000[INTEGER],12]
* \_AggregateExec[[language_code{r}#12],[COUNT(emp_no{r}#31,true[BOOLEAN]) AS c#17, language_code{r}#12],FINAL,[language_code{r}#12
* , $$c$count{r}#32, $$c$seen{r}#33],12]
* \_ExchangeExec[[language_code{r}#12, $$c$count{r}#32, $$c$seen{r}#33],true]
* \_AggregateExec[[language_code{r}#12],[COUNT(emp_no{r}#31,true[BOOLEAN]) AS c#17, language_code{r}#12],INITIAL,[language_code{r}#
- * 12, $$c$count{r}#34, $$c$seen{r}#35],12]
+ * 12, $$c$count{r}#34, $$c$seen{r}#35],12]
* \_LookupJoinExec[[language_code{r}#12],[language_code{f}#29],[]]
- * |_GrokExec[first_name{f}#19,Parser[pattern=%{NUMBER:language_code:int}, grok=org.elasticsearch.grok.Grok@764e5109],[languag
- * e_code{r}#12]]
+ * |_GrokExec[first_name{f}#19,Parser[pattern=%{NUMBER:language_code:int}, grok=org.elasticsearch.grok.Grok@177d8fd5],[languag
+ * e_code{r}#12]]
* | \_MvExpandExec[emp_no{f}#18,emp_no{r}#31]
* | \_ProjectExec[[emp_no{f}#18, languages{r}#21 AS language_code#7, first_name{f}#19]]
* | \_FieldExtractExec[emp_no{f}#18, first_name{f}#19]<[],[]>
* | \_EvalExec[[null[INTEGER] AS languages#21]]
- * | \_EsQueryExec[test], indexMode[standard], query[][_doc{f}#36], limit[], sort[] estimatedRowSize[66]
- * \_EsQueryExec[languages_lookup], indexMode[lookup], query[][_doc{f}#37], limit[], sort[] estimatedRowSize[4]
+ * | \_EsQueryExec[test], indexMode[standard], [_doc{f}#36], limit[], sort[] estimatedRowSize[66]
+ * queryBuilderAndTags [[QueryBuilderAndTags{queryBuilder=[null], tags=[]}]]
+ * \_FragmentExec[filter=null, estimatedRowSize=0, reducer=[], fragment=[<>
+ * EsRelation[languages_lookup][LOOKUP][language_code{f}#29]<>]]
*/
public void testMissingFieldsNotPurgingTheJoinLocally() {
var stats = EsqlTestUtils.statsForMissingField("languages");
@@ -1072,22 +1076,26 @@ public void testMissingFieldsNotPurgingTheJoinLocally() {
var extract = as(project.child(), FieldExtractExec.class);
var eval = as(extract.child(), EvalExec.class);
var source = as(eval.child(), EsQueryExec.class);
- var right = as(join.right(), EsQueryExec.class);
+ var right = as(join.right(), FragmentExec.class);
+ var relation = as(right.fragment(), EsRelation.class);
}
/*
- * LimitExec[1000[INTEGER]]
+ * LimitExec[1000[INTEGER],62]
* \_LookupJoinExec[[language_code{r}#6],[language_code{f}#23],[language_name{f}#24]]
- * |_LimitExec[1000[INTEGER]]
+ * |_LimitExec[1000[INTEGER],12]
* | \_AggregateExec[[languages{f}#15],[COUNT(emp_no{f}#12,true[BOOLEAN]) AS c#10, languages{f}#15 AS language_code#6],FINAL,[language
- * s{f}#15, $$c$count{r}#25, $$c$seen{r}#26],62]
+ * s{f}#15, $$c$count{r}#25, $$c$seen{r}#26],62]
* | \_ExchangeExec[[languages{f}#15, $$c$count{r}#25, $$c$seen{r}#26],true]
* | \_AggregateExec[[languages{r}#15],[COUNT(emp_no{f}#12,true[BOOLEAN]) AS c#10, languages{r}#15 AS language_code#6],INITIAL,
- * [languages{r}#15, $$c$count{r}#27, $$c$seen{r}#28],12]
+ * [languages{r}#15, $$c$count{r}#27, $$c$seen{r}#28],12]
+ * ges{r}#15, $$c$count{r}#27, $$c$seen{r}#28],12]
* | \_FieldExtractExec[emp_no{f}#12]<[],[]>
* | \_EvalExec[[null[INTEGER] AS languages#15]]
- * | \_EsQueryExec[test], indexMode[standard], query[][_doc{f}#29], limit[], sort[] estimatedRowSize[12]
- * \_EsQueryExec[languages_lookup], indexMode[lookup], query[][_doc{f}#30], limit[], sort[] estimatedRowSize[4]
+ * | \_EsQueryExec[test], indexMode[standard], [_doc{f}#29], limit[], sort[] estimatedRowSize[12]
+ * queryBuilderAndTags [[QueryBuilderAndTags{queryBuilder=[null], tags=[]}]]
+ * \_FragmentExec[filter=null, estimatedRowSize=0, reducer=[], fragment=[<>
+ * EsRelation[languages_lookup][LOOKUP][language_code{f}#23, language_name{f}#24]<>]]
*/
public void testMissingFieldsDoesNotPurgeTheJoinOnCoordinator() {
var stats = EsqlTestUtils.statsForMissingField("languages");
@@ -1118,9 +1126,10 @@ public void testMissingFieldsDoesNotPurgeTheJoinOnCoordinator() {
assertThat(source.indexPattern(), is("test"));
assertThat(source.indexMode(), is(IndexMode.STANDARD));
- source = as(join.right(), EsQueryExec.class);
- assertThat(source.indexPattern(), is("languages_lookup"));
- assertThat(source.indexMode(), is(IndexMode.LOOKUP));
+ var right = as(join.right(), FragmentExec.class);
+ var relation = as(right.fragment(), EsRelation.class);
+ assertThat(relation.indexPattern(), is("languages_lookup"));
+ assertThat(relation.indexMode(), is(IndexMode.LOOKUP));
}
/*
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java
index c8a64f779ab4a..790f4b1a9d279 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java
@@ -1032,10 +1032,10 @@ public void testPushdownLimitsPastLeftJoin() {
var rightChild = new LocalRelation(Source.EMPTY, List.of(fieldAttribute()), EmptyLocalSupplier.EMPTY);
assertNotEquals(leftChild, rightChild);
- var joinConfig = new JoinConfig(JoinTypes.LEFT, List.of(), List.of(), List.of());
+ var joinConfig = new JoinConfig(JoinTypes.LEFT, List.of(), List.of(), List.of(), null);
var join = switch (randomIntBetween(0, 2)) {
case 0 -> new Join(EMPTY, leftChild, rightChild, joinConfig);
- case 1 -> new LookupJoin(EMPTY, leftChild, rightChild, joinConfig);
+ case 1 -> new LookupJoin(EMPTY, leftChild, rightChild, joinConfig, false);
case 2 -> new InlineJoin(EMPTY, leftChild, rightChild, joinConfig);
default -> throw new IllegalArgumentException();
};
@@ -7003,13 +7003,14 @@ public void testLookupJoinPushDownFilterOnLeftSideField() {
* Expects
*
* {@code
- * Project[[_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, gender{f}#9, hire_date{f}#14, job{f}#15, job.raw{f}#16,
- * languages{f}#10 AS language_code#4, last_name{f}#11, long_noidx{f}#17, salary{f}#12, language_name{f}#19]]
+ * Project[[_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, gender{f}#9, hire_date{f}#14, job{f}#15, job.raw{f}#16, languages
+ * {f}#10 AS language_code#4, last_name{f}#11, long_noidx{f}#17, salary{f}#12, language_name{f}#19]]
* \_Limit[1000[INTEGER],false]
- * \_Filter[language_name{f}#19 == [45 6e 67 6c 69 73 68][KEYWORD]]
- * \_Join[LEFT,[languages{f}#10],[languages{f}#10],[language_code{f}#18]]
+ * \_Filter[language_name{f}#19 == English[KEYWORD]]
+ * \_Join[LEFT,[languages{f}#10],[languages{f}#10],[language_code{f}#18],false]
* |_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..]
- * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19]
+ * \_Filter[language_name{f}#19 == English[KEYWORD]]
+ * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19]
* }
*/
public void testLookupJoinPushDownDisabledForLookupField() {
@@ -7038,7 +7039,10 @@ public void testLookupJoinPushDownDisabledForLookupField() {
assertThat(join.config().type(), equalTo(JoinTypes.LEFT));
var leftRel = as(join.left(), EsRelation.class);
- var rightRel = as(join.right(), EsRelation.class);
+ var rightFilter = as(join.right(), Filter.class);
+ assertEquals("language_name == \"English\"", rightFilter.condition().toString());
+ var joinRightEsRelation = as(rightFilter.child(), EsRelation.class);
+
}
/**
@@ -7047,14 +7051,15 @@ public void testLookupJoinPushDownDisabledForLookupField() {
* Expects
*
* {@code
- * Project[[_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, gender{f}#10, hire_date{f}#15, job{f}#16, job.raw{f}#17,
- * languages{f}#11 AS language_code#4, last_name{f}#12, long_noidx{f}#18, salary{f}#13, language_name{f}#20]]
+ * Project[[_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, gender{f}#10, hire_date{f}#15, job{f}#16, job.raw{f}#17, languages
+ * {f}#11 AS language_code#4, last_name{f}#12, long_noidx{f}#18, salary{f}#13, language_name{f}#20]]
* \_Limit[1000[INTEGER],false]
- * \_Filter[language_name{f}#20 == [45 6e 67 6c 69 73 68][KEYWORD]]
- * \_Join[LEFT,[languages{f}#11],[languages{f}#11],[language_code{f}#19]]
+ * \_Filter[language_name{f}#20 == English[KEYWORD]]
+ * \_Join[LEFT,[languages{f}#11],[languages{f}#11],[language_code{f}#19],false]
* |_Filter[emp_no{f}#8 > 1[INTEGER]]
* | \_EsRelation[test][_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, ge..]
- * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#19, language_name{f}#20]
+ * \_Filter[language_name{f}#20 == English[KEYWORD]]
+ * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#19, language_name{f}#20]
* }
*/
public void testLookupJoinPushDownSeparatedForConjunctionBetweenLeftAndRightField() {
@@ -7091,7 +7096,9 @@ public void testLookupJoinPushDownSeparatedForConjunctionBetweenLeftAndRightFiel
assertThat(literal.value(), equalTo(1));
var leftRel = as(filter.child(), EsRelation.class);
- var rightRel = as(join.right(), EsRelation.class);
+ var rightFilter = as(join.right(), Filter.class);
+ assertEquals("language_name == \"English\"", rightFilter.condition().toString());
+ var rightRel = as(rightFilter.child(), EsRelation.class);
}
/**
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java
index f3d1107983628..250481f06d3f8 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java
@@ -9,41 +9,56 @@
import org.elasticsearch.index.IndexMode;
import org.elasticsearch.index.query.QueryBuilder;
-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.Expression;
import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
+import org.elasticsearch.xpack.esql.core.expression.FoldContext;
import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute;
import org.elasticsearch.xpack.esql.core.type.DataType;
import org.elasticsearch.xpack.esql.expression.function.aggregate.Count;
import org.elasticsearch.xpack.esql.expression.function.fulltext.Match;
import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow;
+import org.elasticsearch.xpack.esql.expression.function.scalar.nulls.Coalesce;
import org.elasticsearch.xpack.esql.expression.function.scalar.string.regex.RLike;
import org.elasticsearch.xpack.esql.expression.function.scalar.string.regex.WildcardLike;
import org.elasticsearch.xpack.esql.expression.predicate.Predicates;
import org.elasticsearch.xpack.esql.expression.predicate.logical.And;
+import org.elasticsearch.xpack.esql.expression.predicate.logical.Not;
+import org.elasticsearch.xpack.esql.expression.predicate.logical.Or;
+import org.elasticsearch.xpack.esql.expression.predicate.nulls.IsNull;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals;
import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThan;
import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqual;
import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan;
+import org.elasticsearch.xpack.esql.optimizer.AbstractLogicalPlanOptimizerTests;
+import org.elasticsearch.xpack.esql.optimizer.LogicalOptimizerContext;
import org.elasticsearch.xpack.esql.plan.logical.Aggregate;
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.Project;
import org.elasticsearch.xpack.esql.plan.logical.inference.Completion;
import org.elasticsearch.xpack.esql.plan.logical.inference.Rerank;
+import org.elasticsearch.xpack.esql.plan.logical.join.Join;
+import org.elasticsearch.xpack.esql.plan.logical.join.JoinConfig;
+import org.elasticsearch.xpack.esql.plan.logical.join.JoinTypes;
import org.elasticsearch.xpack.esql.plan.logical.local.EsqlProject;
import java.util.ArrayList;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import static java.util.Collections.emptyList;
import static java.util.Collections.singletonList;
+import static org.elasticsearch.xpack.esql.EsqlTestUtils.FIVE;
import static org.elasticsearch.xpack.esql.EsqlTestUtils.FOUR;
import static org.elasticsearch.xpack.esql.EsqlTestUtils.ONE;
+import static org.elasticsearch.xpack.esql.EsqlTestUtils.SIX;
import static org.elasticsearch.xpack.esql.EsqlTestUtils.THREE;
import static org.elasticsearch.xpack.esql.EsqlTestUtils.TWO;
import static org.elasticsearch.xpack.esql.EsqlTestUtils.as;
@@ -56,9 +71,13 @@
import static org.elasticsearch.xpack.esql.EsqlTestUtils.rlike;
import static org.elasticsearch.xpack.esql.EsqlTestUtils.wildcardLike;
import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
import static org.mockito.Mockito.mock;
-public class PushDownAndCombineFiltersTests extends ESTestCase {
+public class PushDownAndCombineFiltersTests extends AbstractLogicalPlanOptimizerTests {
+
+ private final LogicalOptimizerContext optimizerContext = new LogicalOptimizerContext(null, FoldContext.small());
public void testCombineFilters() {
EsRelation relation = relation();
@@ -68,7 +87,10 @@ public void testCombineFilters() {
Filter fa = new Filter(EMPTY, relation, conditionA);
Filter fb = new Filter(EMPTY, fa, conditionB);
- assertEquals(new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)), new PushDownAndCombineFilters().apply(fb));
+ assertEquals(
+ new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)),
+ new PushDownAndCombineFilters().apply(fb, optimizerContext)
+ );
}
public void testCombineFiltersLikeRLike() {
@@ -79,7 +101,10 @@ public void testCombineFiltersLikeRLike() {
Filter fa = new Filter(EMPTY, relation, conditionA);
Filter fb = new Filter(EMPTY, fa, conditionB);
- assertEquals(new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)), new PushDownAndCombineFilters().apply(fb));
+ assertEquals(
+ new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB)),
+ new PushDownAndCombineFilters().apply(fb, optimizerContext)
+ );
}
public void testPushDownFilter() {
@@ -93,7 +118,7 @@ public void testPushDownFilter() {
Filter fb = new Filter(EMPTY, keep, conditionB);
Filter combinedFilter = new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB));
- assertEquals(new EsqlProject(EMPTY, combinedFilter, projections), new PushDownAndCombineFilters().apply(fb));
+ assertEquals(new EsqlProject(EMPTY, combinedFilter, projections), new PushDownAndCombineFilters().apply(fb, optimizerContext));
}
public void testPushDownFilterPastRenamingProject() {
@@ -111,7 +136,7 @@ public void testPushDownFilterPastRenamingProject() {
LessThan bRenamedLessThanTwo = lessThanOf(bRenamed.toAttribute(), TWO);
Filter filter = new Filter(EMPTY, project, Predicates.combineAnd(List.of(aRenamedTwiceGreaterThanOne, bRenamedLessThanTwo)));
- LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
Project optimizedProject = as(optimized, Project.class);
assertEquals(optimizedProject.projections(), project.projections());
@@ -184,7 +209,7 @@ public void testPushDownFilterOnAliasInEval() {
Filter filter = new Filter(EMPTY, eval, Predicates.combineAnd(conditions));
- LogicalPlan plan = new PushDownAndCombineFilters().apply(filter);
+ LogicalPlan plan = new PushDownAndCombineFilters().apply(filter, optimizerContext);
if (numNonPushable > 0) {
Filter optimizedFilter = as(plan, Filter.class);
@@ -216,7 +241,7 @@ public void testPushDownLikeRlikeFilter() {
Filter fb = new Filter(EMPTY, keep, conditionB);
Filter combinedFilter = new Filter(EMPTY, relation, new And(EMPTY, conditionA, conditionB));
- assertEquals(new EsqlProject(EMPTY, combinedFilter, projections), new PushDownAndCombineFilters().apply(fb));
+ assertEquals(new EsqlProject(EMPTY, combinedFilter, projections), new PushDownAndCombineFilters().apply(fb, optimizerContext));
}
// from ... | where a > 1 | stats count(1) by b | where count(1) >= 3 and b < 2
@@ -244,7 +269,7 @@ public void testSelectivelyPushDownFilterPastFunctionAgg() {
),
aggregateCondition
);
- assertEquals(expected, new PushDownAndCombineFilters().apply(fb));
+ assertEquals(expected, new PushDownAndCombineFilters().apply(fb, optimizerContext));
}
// from ... | where a > 1 | COMPLETION completion = "some prompt" WITH { "inferenceId' : "inferenceId" } | where b < 2 and
@@ -284,7 +309,7 @@ public void testPushDownFilterPastCompletion() {
conditionCompletion
);
- assertEquals(expectedOptimizedPlan, new PushDownAndCombineFilters().apply(filterB));
+ assertEquals(expectedOptimizedPlan, new PushDownAndCombineFilters().apply(filterB, optimizerContext));
}
// from ... | where a > 1 | RERANK "query" ON title WITH { "inference_id" : "inferenceId" } | where b < 2 and _score > 1
@@ -317,7 +342,7 @@ public void testPushDownFilterPastRerank() {
scoreCondition
);
- assertEquals(expectedOptimizedPlan, new PushDownAndCombineFilters().apply(filterB));
+ assertEquals(expectedOptimizedPlan, new PushDownAndCombineFilters().apply(filterB, optimizerContext));
}
private static Completion completion(LogicalPlan child) {
@@ -348,4 +373,471 @@ private static EsRelation relation() {
private static EsRelation relation(List fieldAttributes) {
return new EsRelation(EMPTY, randomIdentifier(), randomFrom(IndexMode.values()), Map.of(), fieldAttributes);
}
+
+ public void testPushDownFilterPastLeftJoinWithPushable() {
+ Join join = createLeftJoin();
+ EsRelation left = (EsRelation) join.left();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ // Pushable filter
+ Expression pushableCondition = greaterThanOf(c, ONE);
+ Filter filter = new Filter(EMPTY, join, pushableCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+ // The filter should still be on top
+ Filter topFilter = as(optimized, Filter.class);
+ assertEquals(pushableCondition, topFilter.condition());
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ assertEquals(left, optimizedJoin.left());
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+ assertEquals(pushableCondition, rightFilter.condition());
+ }
+
+ public void testPushDownFilterPastLeftJoinWithExistingFilter() {
+ Join join = createLeftJoin();
+ EsRelation left = (EsRelation) join.left();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ // Existing filter on the right side
+ Expression existingCondition = lessThanOf(c, FIVE);
+ Filter existingFilter = new Filter(EMPTY, join.right(), existingCondition);
+ join = (Join) join.replaceRight(existingFilter);
+
+ // Pushable filter
+ Expression pushableCondition = greaterThanOf(c, ONE);
+ Filter filter = new Filter(EMPTY, join, pushableCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+ // The filter should still be on top
+ Filter topFilter = as(optimized, Filter.class);
+ assertEquals(pushableCondition, topFilter.condition());
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ assertEquals(left, optimizedJoin.left());
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+
+ // The new condition should be merged with the existing one
+ Expression combinedCondition = new And(EMPTY, existingCondition, pushableCondition);
+ assertEquals(combinedCondition, rightFilter.condition());
+
+ // try to apply the filter again, the plan should not change
+ // this verifies that the rule is idempotent
+ // and we will not get in an infinite loop pushing the same filter over and over
+ optimized = new PushDownAndCombineFilters().apply(optimized, optimizerContext);
+
+ topFilter = as(optimized, Filter.class);
+ assertEquals(pushableCondition, topFilter.condition());
+ optimizedJoin = as(topFilter.child(), Join.class);
+ assertEquals(left, optimizedJoin.left());
+ rightFilter = as(optimizedJoin.right(), Filter.class);
+
+ // The new condition should be merged with the existing one
+ assertEquals(combinedCondition, rightFilter.condition());
+
+ }
+
+ public void testDoNotPushDownExistingFilterAgain() {
+ Join join = createLeftJoin();
+ EsRelation left = (EsRelation) join.left();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ // Existing filter on the right side
+ Expression existingCondition = greaterThanOf(c, ONE);
+ Filter existingFilter = new Filter(EMPTY, join.right(), existingCondition);
+ join = (Join) join.replaceRight(existingFilter);
+
+ // A filter on top with the same condition
+ Filter filter = new Filter(EMPTY, join, existingCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+
+ // The filter should still be on top
+ Filter topFilter = as(optimized, Filter.class);
+ assertEquals(existingCondition, topFilter.condition());
+
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ assertEquals(left, optimizedJoin.left());
+
+ // The right side should be the original filter, unchanged.
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+ assertEquals(existingFilter, rightFilter);
+ assertEquals(existingCondition, rightFilter.condition());
+ }
+
+ public void testPushDownFilterPastLeftJoinWithExistingFilterCalledTwice() {
+ Join join = createLeftJoin();
+ EsRelation left = (EsRelation) join.left();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ // Pushable filter
+ Expression pushableCondition = greaterThanOf(c, ONE);
+ Filter filter = new Filter(EMPTY, join, pushableCondition);
+
+ // First optimization
+ LogicalPlan optimizedOnce = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+
+ // Second optimization
+ LogicalPlan optimizedTwice = new PushDownAndCombineFilters().apply(optimizedOnce, optimizerContext);
+
+ // The filter should still be on top
+ Filter topFilter = as(optimizedTwice, Filter.class);
+ assertEquals(pushableCondition, topFilter.condition());
+
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ assertEquals(left, optimizedJoin.left());
+
+ // The right side should have the filter, but not duplicated.
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+ assertEquals(pushableCondition, rightFilter.condition());
+ }
+
+ public void testPushDownFilterPastLeftJoinWithNonPushable() {
+ Join join = createLeftJoin();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ // Non-pushable filter
+ Expression nonPushableCondition = new IsNull(EMPTY, c);
+ Filter filter = new Filter(EMPTY, join, nonPushableCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+ // No optimization should be applied, the plan should be the same
+ assertEquals(filter, optimized);
+ // And the join inside should not have candidate filters
+ Join innerJoin = as(as(optimized, Filter.class).child(), Join.class);
+ assertFalse(innerJoin.right() instanceof Filter);
+ }
+
+ public void testPushDownFilterPastLeftJoinWithPartiallyPushableAnd() {
+ Join join = createLeftJoin();
+ EsRelation left = (EsRelation) join.left();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ Expression pushableCondition = greaterThanOf(c, ONE);
+ Expression nonPushableCondition = new IsNull(EMPTY, c);
+
+ // Partially pushable filter
+ Expression partialCondition = new And(EMPTY, pushableCondition, nonPushableCondition);
+ Filter filter = new Filter(EMPTY, join, partialCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+ Filter topFilter = as(optimized, Filter.class);
+ // The top filter condition should be the original one
+ assertEquals(partialCondition, topFilter.condition());
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ assertEquals(left, optimizedJoin.left());
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+ // Only the pushable part should be a candidate
+ assertEquals(pushableCondition, rightFilter.condition());
+ }
+
+ public void testPushDownFilterPastLeftJoinWithOr() {
+ Join join = createLeftJoin();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ Expression pushableCondition = greaterThanOf(c, ONE);
+ Expression nonPushableCondition = new IsNull(EMPTY, c);
+
+ // OR of pushable and non-pushable filter
+ Expression orCondition = new Or(EMPTY, pushableCondition, nonPushableCondition);
+ Filter filter = new Filter(EMPTY, join, orCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+ // No optimization should be applied, the plan should be the same
+ assertEquals(filter, optimized);
+ // And the join inside should not have candidate filters
+ Join innerJoin = as(filter.child(), Join.class);
+ assertFalse(innerJoin.right() instanceof Filter);
+ }
+
+ public void testPushDownFilterPastLeftJoinWithNotButStillPushable() {
+ Join join = createLeftJoin();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ Expression pushableCondition = greaterThanOf(c, ONE);
+
+ // negation of pushable filter, in this case it remains pushable
+ Expression negationOfPushableCondition = new Not(EMPTY, pushableCondition);
+ Filter filter = new Filter(EMPTY, join, negationOfPushableCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+ Filter topFilter = as(optimized, Filter.class);
+ assertEquals(negationOfPushableCondition, topFilter.condition());
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+ assertEquals(negationOfPushableCondition, rightFilter.condition());
+ }
+
+ public void testPushDownFilterPastLeftJoinWithNotNonPushable() {
+ Join join = createLeftJoin();
+ FieldAttribute c = (FieldAttribute) join.right().output().get(0);
+
+ Expression nonPushableCondition = new IsNull(EMPTY, c);
+
+ // negation of non-pushable filter makes it pushable
+ Expression negationOfNonPushableCondition = new Not(EMPTY, nonPushableCondition);
+ Filter filter = new Filter(EMPTY, join, negationOfNonPushableCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+ Filter topFilter = as(optimized, Filter.class);
+ assertEquals(negationOfNonPushableCondition, topFilter.condition());
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+ assertEquals(negationOfNonPushableCondition, rightFilter.condition());
+ }
+
+ public void testPushDownFilterPastLeftJoinWithComplexMix() {
+ // Setup
+ FieldAttribute a = getFieldAttribute("a");
+ FieldAttribute c = getFieldAttribute("c");
+ FieldAttribute d = getFieldAttribute("d");
+ FieldAttribute e = getFieldAttribute("e");
+ FieldAttribute f = getFieldAttribute("f");
+ FieldAttribute g = getFieldAttribute("g");
+ EsRelation left = relation(List.of(a, getFieldAttribute("b")));
+ EsRelation right = relation(List.of(c, d, e, f, g));
+ JoinConfig joinConfig = new JoinConfig(JoinTypes.LEFT, List.of(a), List.of(a), List.of(c), null);
+ Join join = new Join(EMPTY, left, right, joinConfig);
+
+ // Predicates
+ Expression p1 = greaterThanOf(c, ONE); // pushable
+ Expression p2 = new Not(EMPTY, new IsNull(EMPTY, d)); // pushable (d IS NOT NULL)
+ Expression p3 = lessThanOf(e, THREE); // pushable
+ Expression p4 = rlike(f, "pat"); // pushable
+ Expression p5 = new Not(EMPTY, new IsNull(EMPTY, g)); // pushable (g IS NOT NULL)
+ Expression p6 = greaterThanOf(c, TWO); // pushable
+ Expression p7 = lessThanOf(d, FOUR); // pushable
+ Expression p8 = greaterThanOf(e, FIVE); // pushable
+
+ Expression np1 = new IsNull(EMPTY, c); // non-pushable (c IS NULL)
+ Expression np2 = new Equals(EMPTY, new Coalesce(EMPTY, d, List.of(SIX)), SIX); // non-pushable
+
+ // Build a complex condition
+ // np2 AND ((p1 AND p2 AND p3 AND p4 AND p5) AND (np1 OR (p6 AND p7) OR (p8 AND np2))) AND p1 AND p6
+ Expression pushableBranch = Predicates.combineAnd(List.of(p1, p2, p3, p4, p5));
+ Expression nonPushableBranch = new Or(EMPTY, np1, new Or(EMPTY, new And(EMPTY, p6, p7), new And(EMPTY, p8, np2)));
+ Expression complexCondition = new And(EMPTY, pushableBranch, nonPushableBranch);
+ complexCondition = Predicates.combineAnd(List.of(np2, complexCondition, p1, p6));
+
+ Filter filter = new Filter(EMPTY, join, complexCondition);
+ LogicalPlan optimized = new PushDownAndCombineFilters().apply(filter, optimizerContext);
+
+ // The top filter with the original condition should remain, but the structure of the AND tree might have changed.
+ // So, we flatten the conditions and compare them as a set.
+ Filter topFilter = as(optimized, Filter.class);
+ Set actualTopPredicates = new HashSet<>(Predicates.splitAnd(topFilter.condition()));
+ Set expectedTopPredicates = new HashSet<>(List.of(p1, p2, p3, p4, p5, nonPushableBranch, np2, p1, p6));
+ assertEquals(expectedTopPredicates, actualTopPredicates);
+
+ // The pushable part of the filter should be added as a candidate to the join
+ Join optimizedJoin = as(topFilter.child(), Join.class);
+ assertEquals(left, optimizedJoin.left());
+ Filter rightFilter = as(optimizedJoin.right(), Filter.class);
+ Set actualPushable = new HashSet<>(Predicates.splitAnd(rightFilter.condition()));
+ Set expectedPushable = new HashSet<>(List.of(p1, p2, p3, p4, p5, p6));
+ assertEquals(expectedPushable, actualPushable);
+ }
+
+ /**
+ * Project[[_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, gender{f}#9, hire_date{f}#14, job{f}#15, job.raw{f}#16, languages
+ * {f}#10 AS language_code#4, last_name{f}#11, long_noidx{f}#17, salary{f}#12, language_name{f}#19]]
+ * \_Limit[1000[INTEGER],false]
+ * \_Filter[ISNULL(language_name{f}#19)]
+ * \_Join[LEFT,[languages{f}#10],[languages{f}#10],[language_code{f}#18]]
+ * |_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..]
+ * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19]
+ */
+ public void testDoNotPushDownIsNullFilterPastLookupJoin() {
+ var plan = plan("""
+ FROM test
+ | RENAME languages AS language_code
+ | LOOKUP JOIN languages_lookup ON language_code
+ | WHERE language_name IS NULL
+ """);
+
+ var project = as(plan, Project.class);
+ var limit = as(project.child(), Limit.class);
+ var filter = as(limit.child(), Filter.class);
+ var join = as(filter.child(), Join.class);
+ assertThat(join.right(), instanceOf(EsRelation.class));
+ }
+
+ /**
+ * Project[[_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, gender{f}#9, hire_date{f}#14, job{f}#15, job.raw{f}#16, languages
+ * {f}#10 AS language_code#4, last_name{f}#11, long_noidx{f}#17, salary{f}#12, language_name{f}#19]]
+ * \_Limit[1000[INTEGER],false]
+ * \_Filter[language_name{f}#19 > a[KEYWORD]]
+ * \_Join[LEFT,[languages{f}#10],[languages{f}#10],[language_code{f}#18],false]
+ * |_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..]
+ * \_Filter[language_name{f}#19 > a[KEYWORD]]
+ * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19]
+ */
+ public void testPushDownGreaterThanFilterPastLookupJoin() {
+ var plan = plan("""
+ FROM test
+ | RENAME languages AS language_code
+ | LOOKUP JOIN languages_lookup ON language_code
+ | WHERE language_name > "a"
+ """);
+
+ var project = as(plan, Project.class);
+ var limit = as(project.child(), Limit.class);
+ var filter = as(limit.child(), Filter.class);
+ var join = as(filter.child(), Join.class);
+ var rightFilter = as(join.right(), Filter.class);
+ assertThat(rightFilter.condition().toString(), is("language_name > \"a\""));
+ }
+
+ /**
+ * Project[[_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, gender{f}#9, hire_date{f}#14, job{f}#15, job.raw{f}#16, languages
+ * {f}#10 AS language_code#4, last_name{f}#11, long_noidx{f}#17, salary{f}#12, language_name{f}#19]]
+ * \_Limit[1000[INTEGER],false]
+ * \_Filter[COALESCE(language_name{f}#19,a[KEYWORD]) == a[KEYWORD]]
+ * \_Join[LEFT,[languages{f}#10],[languages{f}#10],[language_code{f}#18]]
+ * |_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..]
+ * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19]
+ */
+ public void testDoNotPushDownCoalesceFilterPastLookupJoin() {
+ var plan = plan("""
+ FROM test
+ | RENAME languages AS language_code
+ | LOOKUP JOIN languages_lookup ON language_code
+ | WHERE COALESCE(language_name, "a") == "a"
+ """);
+
+ var project = as(plan, Project.class);
+ var limit = as(project.child(), Limit.class);
+ var filter = as(limit.child(), Filter.class);
+ var join = as(filter.child(), Join.class);
+ assertThat(join.right(), instanceOf(EsRelation.class));
+ }
+
+ /**
+ *
+ * Project[[_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, gender{f}#9, hire_date{f}#14, job{f}#15, job.raw{f}#16, languages
+ * {f}#10 AS language_code#4, last_name{f}#11, long_noidx{f}#17, salary{f}#12, language_name{f}#19]]
+ * \_Limit[1000[INTEGER],false]
+ * \_Filter[ISNOTNULL(language_name{f}#19)]
+ * \_Join[LEFT,[languages{f}#10],[languages{f}#10],[language_code{f}#18],false]
+ * |_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..]
+ * \_Filter[ISNOTNULL(language_name{f}#19)]
+ * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19]
+ */
+ public void testPushDownIsNotNullFilterPastLookupJoin() {
+ var plan = plan("""
+ FROM test
+ | RENAME languages AS language_code
+ | LOOKUP JOIN languages_lookup ON language_code
+ | WHERE language_name IS NOT NULL
+ """);
+
+ var project = as(plan, Project.class);
+ var limit = as(project.child(), Limit.class);
+ var filter = as(limit.child(), Filter.class);
+ var join = as(filter.child(), Join.class);
+ var rightFilter = as(join.right(), Filter.class);
+ assertThat(rightFilter.condition().toString(), is("language_name IS NOT NULL"));
+ }
+
+ /**
+ * Project[[_meta_field{f}#17, emp_no{f}#11, first_name{f}#12, gender{f}#13, hire_date{f}#18, job{f}#19, job.raw{f}#20, languages
+ * {f}#14 AS language_code#4, last_name{f}#15, long_noidx{f}#21, salary{f}#16, language_name{f}#23]]
+ * \_Limit[1000[INTEGER],false]
+ * \_Filter[ISNOTNULL(language_name{f}#23) AND language_name{f}#23 > a[KEYWORD] AND LIKE(language_name{f}#23, "*b", false)
+ * AND COALESCE(language_name{f}#23,c[KEYWORD]) == c[KEYWORD] AND RLIKE(language_name{f}#23, "f.*", false)]
+ * \_Join[LEFT,[languages{f}#14],[languages{f}#14],[language_code{f}#22]]
+ * |_EsRelation[test][_meta_field{f}#17, emp_no{f}#11, first_name{f}#12, ..]
+ * \_Filter[ISNOTNULL(language_name{f}#23) AND language_name{f}#23 > a[KEYWORD] AND LIKE(language_name{f}#23, "*b", false)
+ * AND RLIKE(language_name{f}#23, "f.*", false)]
+ * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#22, language_name{f}#23]
+ */
+ public void testPushDownMultipleWhere() {
+ var plan = plan("""
+ FROM test
+ | RENAME languages AS language_code
+ | LOOKUP JOIN languages_lookup ON language_code
+ | WHERE language_name IS NOT NULL
+ | WHERE language_name > "a"
+ | WHERE language_name LIKE "*b"
+ | WHERE COALESCE(language_name, "c") == "c"
+ | WHERE language_name RLIKE "f.*"
+ """);
+
+ var project = as(plan, Project.class);
+ var limit = as(project.child(), Limit.class);
+ var topFilter = as(limit.child(), Filter.class);
+
+ // Verify the top-level filter contains all 5 original conditions combined
+ Set expectedAllFilters = Set.of(
+ "language_name IS NOT NULL",
+ "language_name > \"a\"",
+ "language_name LIKE \"*b\"",
+ "COALESCE(language_name, \"c\") == \"c\"",
+ "language_name RLIKE \"f.*\""
+ );
+
+ Set actualAllFilters = new HashSet<>(Predicates.splitAnd(topFilter.condition()).stream().map(Object::toString).toList());
+ assertEquals(expectedAllFilters, actualAllFilters);
+
+ // Verify the join is below the top-level filter
+ var join = as(topFilter.child(), Join.class);
+
+ // Verify a new filter with only the pushable predicates has been pushed to the right side of the join
+ var rightFilter = as(join.right(), Filter.class);
+ Set expectedPushedFilters = Set.of(
+ "language_name IS NOT NULL",
+ "language_name > \"a\"",
+ "language_name LIKE \"*b\"",
+ "language_name RLIKE \"f.*\""
+ );
+ Set actualPushedFilters = new HashSet<>(
+ Predicates.splitAnd(rightFilter.condition()).stream().map(Object::toString).toList()
+ );
+ assertEquals(expectedPushedFilters, actualPushedFilters);
+ }
+
+ /**
+ *
+ * Project[[$$languages$temp_name$32{r$}#33 AS language_code#4, salary{f}#13, language_name{f}#20, _meta_field{f}#27, emp
+ * _no{f}#21, first_name{f}#22, gender{f}#23, hire_date{f}#28, job{f}#29, job.raw{f}#30, languages{f}#24,
+ * last_name{f}#25, long_noidx{f}#31]]
+ * \_Limit[1000[INTEGER],true]
+ * \_Join[LEFT,[salary{f}#13],[salary{f}#13],[salary{f}#26]]
+ * |_Eval[[languages{f}#11 AS $$languages$temp_name$32#33]]
+ * | \_Limit[1000[INTEGER],false]
+ * | \_Filter[language_name{f}#20 > a[KEYWORD]]
+ * | \_Join[LEFT,[languages{f}#11],[languages{f}#11],[language_code{f}#19]]
+ * | |_EsRelation[test][_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, ge..]
+ * | \_Filter[language_name{f}#20 > a[KEYWORD]]
+ * | \_EsRelation[languages_lookup][LOOKUP][language_code{f}#19, language_name{f}#20]
+ * \_EsRelation[test_lookup][LOOKUP][_meta_field{f}#27, emp_no{f}#21, first_name{f}#22, ..]
+ */
+ public void testPushDownFilterPastTwoLookupJoins() {
+ var plan = plan("""
+ FROM test
+ | RENAME languages AS language_code
+ | LOOKUP JOIN languages_lookup ON language_code
+ | LOOKUP JOIN test_lookup ON salary
+ | WHERE language_name > "a"
+ """);
+
+ var project = as(plan, Project.class);
+ var limit = as(project.child(), Limit.class);
+
+ // The filter is pushed down past the top join, so a Join is now at the top of the plan after the limit
+ var topJoin = as(limit.child(), Join.class);
+ assertThat(topJoin.right(), instanceOf(EsRelation.class)); // No filter on the top lookup join's right side
+
+ // Traverse down the left side of the top join to find the filter and the bottom join
+ var eval = as(topJoin.left(), Eval.class);
+ var innerLimit = as(eval.child(), Limit.class);
+ var topFilter = as(innerLimit.child(), Filter.class);
+ assertThat(topFilter.condition().toString(), is("language_name > \"a\""));
+
+ // make sure that the filter was pushed to the right side of the bottom join
+ var bottomJoin = as(topFilter.child(), Join.class);
+ var rightFilter = as(bottomJoin.right(), Filter.class);
+ assertThat(rightFilter.condition().toString(), is("language_name > \"a\""));
+ }
+
+ private Join createLeftJoin() {
+ FieldAttribute a = getFieldAttribute("a");
+ FieldAttribute b = getFieldAttribute("b");
+ FieldAttribute c = getFieldAttribute("c");
+ EsRelation left = relation(List.of(a, b));
+ EsRelation right = relation(List.of(c, b));
+
+ JoinConfig joinConfig = new JoinConfig(JoinTypes.LEFT, List.of(b), List.of(a, b), List.of(b, c), null);
+ return new Join(EMPTY, left, right, joinConfig);
+ }
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java
index 96f63d0d7eb81..746f74bb307be 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownJoinPastProjectTests.java
@@ -240,6 +240,197 @@ public void testShadowingAfterPushdown2() {
assertLeftJoinConfig(join.config(), "emp_no", mainRel.outputSet(), "languages", lookupRel.outputSet());
}
+ /**
+ * EsqlProject[[languages{f}#30, emp_no{f}#27, salary{f}#32]]
+ * \_Limit[1000[INTEGER],true]
+ * \_Join[LEFT,[lang{r}#10, languages{f}#30],[lang{r}#10],[languages{f}#30],lang{r}#10 == languages{f}#30]
+ * |_EsqlProject[[_meta_field{f}#22, emp_no{f}#16, first_name{f}#17, gender{f}#18, hire_date{f}#23, job{f}#24, job.raw{f}#25,
+ * languages{f}#19 AS lang2#4, last_name{f}#20, long_noidx{f}#26, salary{f}#21, emp_no{f}#16 AS lang#10]]
+ * | \_Limit[1000[INTEGER],false]
+ * | \_EsRelation[test][_meta_field{f}#22, emp_no{f}#16, first_name{f}#17, ..]
+ * \_EsRelation[test_lookup][LOOKUP][emp_no{f}#27, languages{f}#30, salary{f}#32]
+ */
+ public void testShadowingAfterPushdownExpressionJoin() {
+ assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled());
+
+ String query = """
+ FROM test_lookup
+ | RENAME languages as lang2
+ | EVAL y = emp_no
+ | RENAME y AS lang
+ | LOOKUP JOIN test_lookup ON lang == languages
+ | KEEP languages, emp_no, salary
+ """;
+
+ var plan = optimizedPlan(query);
+ var project = as(plan, Project.class);
+ var limit1 = asLimit(project.child(), 1000, true);
+ var join = as(limit1.child(), Join.class);
+ var lookupRel = as(join.right(), EsRelation.class);
+ var project2 = as(join.left(), Project.class);
+ var limit2 = asLimit(project2.child(), 1000, false);
+ var mainRel = as(limit2.child(), EsRelation.class);
+
+ var projections = project.projections();
+ assertThat(Expressions.names(projections), contains("languages", "emp_no", "salary"));
+
+ var languages = as(projections.get(0), FieldAttribute.class);
+ assertEquals("languages", languages.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(languages));
+
+ var empNo = as(projections.get(1), FieldAttribute.class);
+ assertEquals("emp_no", empNo.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(empNo));
+
+ var salary = as(projections.get(2), FieldAttribute.class);
+ assertEquals("salary", salary.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(salary));
+
+ var project2Projections = project2.projections();
+ var lang = as(project2Projections.stream().filter(p -> "lang".equals(p.name())).findFirst().get(), Alias.class);
+ var langAttr = lang.toAttribute();
+ var originalEmpNo = as(lang.child(), FieldAttribute.class);
+ assertEquals("emp_no", originalEmpNo.fieldName().string());
+ assertTrue(mainRel.outputSet().contains(originalEmpNo));
+
+ var joinConfig = join.config();
+ assertSame(JoinTypes.LEFT, joinConfig.type());
+ var leftKeys = joinConfig.leftFields();
+ assertEquals(1, leftKeys.size());
+ assertTrue(leftKeys.get(0).semanticEquals(langAttr));
+ var rightKeys = joinConfig.rightFields();
+ assertEquals(1, rightKeys.size());
+ assertEquals("languages", rightKeys.get(0).name());
+ assertTrue(lookupRel.outputSet().contains(rightKeys.get(0)));
+ }
+
+ /**
+ * EsqlProject[[languages{f}#24, emp_no{f}#21, salary{f}#26]]
+ * \_Limit[1000[INTEGER],true]
+ * \_Join[LEFT,[lang{r}#4, languages{f}#24],[lang{r}#4],[languages{f}#24],lang{r}#4 == languages{f}#24]
+ * |_EsqlProject[[_meta_field{f}#16, emp_no{f}#10, first_name{f}#11, gender{f}#12, hire_date{f}#17, job{f}#18, job.raw{f}#19,
+ * languages{f}#13 AS lang#4, last_name{f}#14, long_noidx{f}#20, salary{f}#15]]
+ * | \_Limit[1000[INTEGER],false]
+ * | \_EsRelation[test][_meta_field{f}#16, emp_no{f}#10, first_name{f}#11, ..]
+ * \_EsRelation[test_lookup][LOOKUP][emp_no{f}#21, languages{f}#24, salary{f}#26]
+ */
+ public void testShadowingAfterPushdownRenameExpressionJoin() {
+ assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled());
+
+ String query = """
+ FROM test_lookup
+ | RENAME languages AS lang
+ | LOOKUP JOIN test_lookup ON lang == languages
+ | KEEP languages, emp_no, salary
+ """;
+
+ var plan = optimizedPlan(query);
+ var project = as(plan, Project.class);
+ var limit1 = asLimit(project.child(), 1000, true);
+ var join = as(limit1.child(), Join.class);
+ var lookupRel = as(join.right(), EsRelation.class);
+ var project2 = as(join.left(), Project.class);
+ var limit2 = asLimit(project2.child(), 1000, false);
+ var mainRel = as(limit2.child(), EsRelation.class);
+
+ var projections = project.projections();
+ assertThat(Expressions.names(projections), contains("languages", "emp_no", "salary"));
+
+ var languages = as(projections.get(0), FieldAttribute.class);
+ assertEquals("languages", languages.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(languages));
+
+ var empNo = as(projections.get(1), FieldAttribute.class);
+ assertEquals("emp_no", empNo.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(empNo));
+
+ var salary = as(projections.get(2), FieldAttribute.class);
+ assertEquals("salary", salary.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(salary));
+
+ var project2Projections = project2.projections();
+ var lang = as(project2Projections.stream().filter(p -> "lang".equals(p.name())).findFirst().get(), Alias.class);
+ var langAttr = lang.toAttribute();
+ var originalLanguages = as(lang.child(), FieldAttribute.class);
+ assertEquals("languages", originalLanguages.fieldName().string());
+ assertTrue(mainRel.outputSet().contains(originalLanguages));
+
+ var joinConfig = join.config();
+ assertSame(JoinTypes.LEFT, joinConfig.type());
+ var leftKeys = joinConfig.leftFields();
+ assertEquals(1, leftKeys.size());
+ assertTrue(leftKeys.get(0).semanticEquals(langAttr));
+ var rightKeys = joinConfig.rightFields();
+ assertEquals(1, rightKeys.size());
+ assertEquals("languages", rightKeys.get(0).name());
+ assertTrue(lookupRel.outputSet().contains(rightKeys.get(0)));
+ }
+
+ /**
+ * EsqlProject[[languages{f}#25, emp_no{f}#22, salary{f}#27]]
+ * \_Limit[1000[INTEGER],true]
+ * \_Join[LEFT,[languages{f}#25, lang{r}#4],[lang{r}#4],[languages{f}#25],lang{r}#4 == languages{f}#25]
+ * |_EsqlProject[[_meta_field{f}#17, emp_no{f}#11, first_name{f}#12, gender{f}#13, hire_date{f}#18, job{f}#19, job.raw{f}#20, l
+ * ast_name{f}#15, long_noidx{f}#21, salary{f}#16, lang{r}#4]]
+ * | \_Eval[[languages{f}#14 + 0[INTEGER] AS lang#4]]
+ * | \_Limit[1000[INTEGER],false]
+ * | \_EsRelation[test][_meta_field{f}#17, emp_no{f}#11, first_name{f}#12, ..]
+ * \_EsRelation[test_lookup][LOOKUP][emp_no{f}#22, languages{f}#25, salary{f}#27]
+ */
+ public void testShadowingAfterPushdownEvalExpressionJoin() {
+ assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled());
+
+ String query = """
+ FROM test_lookup
+ | EVAL lang = languages + 0
+ | DROP languages
+ | LOOKUP JOIN test_lookup ON lang == languages
+ | KEEP languages, emp_no, salary
+ """;
+
+ var plan = optimizedPlan(query);
+ var project = as(plan, Project.class);
+ var limit1 = asLimit(project.child(), 1000, true);
+ var join = as(limit1.child(), Join.class);
+ var lookupRel = as(join.right(), EsRelation.class);
+ var project2 = as(join.left(), Project.class);
+ var eval = as(project2.child(), Eval.class);
+ var limit2 = asLimit(eval.child(), 1000, false);
+ var mainRel = as(limit2.child(), EsRelation.class);
+
+ var projections = project.projections();
+ assertThat(Expressions.names(projections), contains("languages", "emp_no", "salary"));
+
+ var languages = as(projections.get(0), FieldAttribute.class);
+ assertEquals("languages", languages.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(languages));
+
+ var empNo = as(projections.get(1), FieldAttribute.class);
+ assertEquals("emp_no", empNo.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(empNo));
+
+ var salary = as(projections.get(2), FieldAttribute.class);
+ assertEquals("salary", salary.fieldName().string());
+ assertTrue(lookupRel.outputSet().contains(salary));
+
+ var lang = as(eval.fields().get(0), Alias.class);
+ var langAttr = lang.toAttribute();
+ var add = as(lang.child(), org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add.class);
+ var originalLanguages = as(add.left(), FieldAttribute.class);
+ assertEquals("languages", originalLanguages.fieldName().string());
+ assertTrue(mainRel.outputSet().contains(originalLanguages));
+
+ var joinConfig = join.config();
+ assertSame(JoinTypes.LEFT, joinConfig.type());
+ var leftKeys = joinConfig.leftFields();
+ assertEquals(1, leftKeys.size());
+ assertTrue(leftKeys.get(0).semanticEquals(langAttr));
+ var rightKeys = joinConfig.rightFields();
+ assertEquals(1, rightKeys.size());
+ assertEquals("languages", rightKeys.get(0).name());
+ assertTrue(lookupRel.outputSet().contains(rightKeys.get(0)));
+ }
+
private static void assertLeftJoinConfig(
JoinConfig config,
String expectedLeftFieldName,
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceRoundToWithQueryAndTagsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceRoundToWithQueryAndTagsTests.java
index 79c484c916af5..beef7ee4857a3 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceRoundToWithQueryAndTagsTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceRoundToWithQueryAndTagsTests.java
@@ -29,11 +29,13 @@
import org.elasticsearch.xpack.esql.expression.function.scalar.math.RoundTo;
import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalPlanOptimizerTests;
import org.elasticsearch.xpack.esql.optimizer.TestPlannerOptimizer;
+import org.elasticsearch.xpack.esql.plan.logical.EsRelation;
import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
import org.elasticsearch.xpack.esql.plan.physical.EsQueryExec;
import org.elasticsearch.xpack.esql.plan.physical.EvalExec;
import org.elasticsearch.xpack.esql.plan.physical.ExchangeExec;
import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec;
+import org.elasticsearch.xpack.esql.plan.physical.FragmentExec;
import org.elasticsearch.xpack.esql.plan.physical.LimitExec;
import org.elasticsearch.xpack.esql.plan.physical.LookupJoinExec;
import org.elasticsearch.xpack.esql.plan.physical.MergeExec;
@@ -295,7 +297,24 @@ public void testDateTruncBucketTransformToQueryAndTagsWithOtherPushdownFunctions
}
}
- // ReplaceRoundToWithQueryAndTags does not support lookup joins yet
+ /**
+ * ReplaceRoundToWithQueryAndTags does not support lookup joins yet
+ * LimitExec[1000[INTEGER],16]
+ * \_AggregateExec[[x{r}#8],[COUNT(*[KEYWORD],true[BOOLEAN]) AS count(*)#9, x{r}#8],FINAL,[x{r}#8, $$count(*)$count{r}#34, $$count(*
+ * )$seen{r}#35],16]
+ * \_ExchangeExec[[x{r}#8, $$count(*)$count{r}#34, $$count(*)$seen{r}#35],true]
+ * \_AggregateExec[[x{r}#8],[COUNT(*[KEYWORD],true[BOOLEAN]) AS count(*)#9, x{r}#8],INITIAL,[x{r}#8, $$count(*)$count{r}#36, $$count
+ * (*)$seen{r}#37],16]
+ * \_EvalExec[[ROUNDTO(date{f}#15,1697760000000[DATETIME],1697846400000[DATETIME],1697932800000[DATETIME],1698019200000[DATE
+ * TIME]) AS x#8]]
+ * \_FieldExtractExec[date{f}#15]
+ * \_LookupJoinExec[[integer{f}#21],[language_code{f}#32],[]]
+ * |_FieldExtractExec[integer{f}#21]
+ * | \_EsQueryExec[test], indexMode[standard], [_doc{f}#38], limit[], sort[] estimatedRowSize[24]
+ * queryBuilderAndTags [[QueryBuilderAndTags{queryBuilder=[null], tags=[]}]]
+ * \_FragmentExec[filter=null, estimatedRowSize=0, reducer=[], fragment=[
+ * EsRelation[languages_lookup][LOOKUP][language_code{f}#32]]]
+ */
public void testDateTruncBucketNotTransformToQueryAndTagsWithLookupJoin() {
for (String dateHistogram : dateHistograms) {
String query = LoggerMessageFormat.format(null, """
@@ -341,14 +360,9 @@ public void testDateTruncBucketNotTransformToQueryAndTagsWithLookupJoin() {
assertTrue(queryBuilder.tags().isEmpty());
assertNull(esQueryExec.query());
// rhs of lookup join
- esQueryExec = as(lookupJoinExec.right(), EsQueryExec.class);
- assertEquals("languages_lookup", esQueryExec.indexPattern());
- queryBuilderAndTags = esQueryExec.queryBuilderAndTags();
- assertEquals(1, queryBuilderAndTags.size());
- queryBuilder = queryBuilderAndTags.get(0);
- assertNull(queryBuilder.query());
- assertTrue(queryBuilder.tags().isEmpty());
- assertNull(esQueryExec.query());
+ FragmentExec fragmentExec = as(lookupJoinExec.right(), FragmentExec.class);
+ EsRelation esRelation = as(fragmentExec.fragment(), EsRelation.class);
+ assertTrue(esRelation.toString().contains("EsRelation[languages_lookup][LOOKUP]"));
}
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java
index e1c52eb6845d9..bb42c8d4884ec 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/parser/StatementParserTests.java
@@ -4638,9 +4638,7 @@ public void testMixedSingleDoubleParams() {
expectError(
LoggerMessageFormat.format(null, "from test | " + command, param1, param2, param3),
List.of(paramAsConstant("f1", "f1"), paramAsConstant("f2", "f2"), paramAsConstant("f3", "f3")),
- command.contains("join")
- ? "JOIN ON clause only supports fields at the moment"
- : "declared as a constant, cannot be used as an identifier"
+ "declared as a constant, cannot be used as an identifier"
);
}
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/CommandLicenseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/CommandLicenseTests.java
index e55a04c03cd1b..204aee977f5c2 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/CommandLicenseTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/CommandLicenseTests.java
@@ -178,7 +178,7 @@ private static LogicalPlan createInstance(Class extends LogicalPlan> clazz, Lo
return new Sample(source, null, child);
}
case "LookupJoin" -> {
- return new LookupJoin(source, child, child, List.of());
+ return new LookupJoin(source, child, child, List.of(), false, null);
}
case "Limit" -> {
return new Limit(source, null, child);
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinSerializationTests.java
index 2b812e4caf260..c88ef47a72989 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinSerializationTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinSerializationTests.java
@@ -32,7 +32,7 @@ private static JoinConfig randomJoinConfig() {
List matchFields = randomFieldAttributes(1, 10, false);
List leftFields = randomFieldAttributes(1, 10, false);
List rightFields = randomFieldAttributes(1, 10, false);
- return new JoinConfig(type, matchFields, leftFields, rightFields);
+ return new JoinConfig(type, matchFields, leftFields, rightFields, null);
}
@Override
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinTests.java
index 62c4812d157e5..21f6f5cd49f43 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/JoinTests.java
@@ -47,7 +47,7 @@ public void testExpressionsAndReferences() {
Row left = new Row(Source.EMPTY, leftFields);
Row right = new Row(Source.EMPTY, rightFields);
- JoinConfig joinConfig = new JoinConfig(JoinTypes.LEFT, matchFields, leftAttributes, rightAttributes);
+ JoinConfig joinConfig = new JoinConfig(JoinTypes.LEFT, matchFields, leftAttributes, rightAttributes, null);
Join join = new Join(Source.EMPTY, left, right, joinConfig);
// matchfields are a subset of the left and right fields, so they don't contribute to the size of the references set.
@@ -87,7 +87,7 @@ public void testTransformExprs() {
Row left = new Row(Source.EMPTY, leftFields);
Row right = new Row(Source.EMPTY, rightFields);
- JoinConfig joinConfig = new JoinConfig(JoinTypes.LEFT, matchFields, leftAttributes, rightAttributes);
+ JoinConfig joinConfig = new JoinConfig(JoinTypes.LEFT, matchFields, leftAttributes, rightAttributes, null);
Join join = new Join(Source.EMPTY, left, right, joinConfig);
assertTrue(join.config().matchFields().stream().allMatch(ref -> ref.dataType().equals(DataType.INTEGER)));
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/LookupSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/LookupSerializationTests.java
index 53f36e124ebb0..8ce5b8c06f052 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/LookupSerializationTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/LookupSerializationTests.java
@@ -24,7 +24,7 @@ public static Lookup randomLookup(int depth) {
Expression tableName = AbstractExpressionSerializationTests.randomChild();
List matchFields = randomFieldAttributes(1, 10, false);
LocalRelation localRelation = randomBoolean() ? null : LocalRelationSerializationTests.randomLocalRelation();
- return new Lookup(source, child, tableName, matchFields, localRelation);
+ return new Lookup(source, child, tableName, matchFields, localRelation, null);
}
@Override
@@ -48,6 +48,6 @@ protected Lookup mutateInstance(Lookup instance) throws IOException {
() -> randomBoolean() ? null : LocalRelationSerializationTests.randomLocalRelation()
);
}
- return new Lookup(source, child, tableName, matchFields, localRelation);
+ return new Lookup(source, child, tableName, matchFields, localRelation, null);
}
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/AbstractPhysicalPlanSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/AbstractPhysicalPlanSerializationTests.java
index ac831d36f1533..c2a012cd8e853 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/AbstractPhysicalPlanSerializationTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/AbstractPhysicalPlanSerializationTests.java
@@ -53,6 +53,7 @@ protected final NamedWriteableRegistry getNamedWriteableRegistry() {
entries.addAll(new SearchModule(Settings.EMPTY, List.of()).getNamedWriteables()); // Query builders
entries.add(Add.ENTRY); // Used by the eval tests
entries.add(AggregateMetricDoubleBlockBuilder.AggregateMetricDoubleLiteral.ENTRY);
+ entries.add(LookupJoinExec.ENTRY);
return new NamedWriteableRegistry(entries);
}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExecSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExecSerializationTests.java
new file mode 100644
index 0000000000000..ac94839350e14
--- /dev/null
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExecSerializationTests.java
@@ -0,0 +1,58 @@
+/*
+ * 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.plan.physical;
+
+import org.elasticsearch.xpack.esql.core.expression.Attribute;
+import org.elasticsearch.xpack.esql.core.tree.Source;
+
+import java.io.IOException;
+import java.util.List;
+
+public class LookupJoinExecSerializationTests extends AbstractPhysicalPlanSerializationTests {
+ public static LookupJoinExec randomLookupJoinExec(int depth) {
+ Source source = randomSource();
+ PhysicalPlan child = randomChild(depth);
+ PhysicalPlan lookup = randomChild(depth);
+ List leftFields = randomFields();
+ List rightFields = randomFields();
+ List addedFields = randomFields();
+ return new LookupJoinExec(source, child, lookup, leftFields, rightFields, addedFields, null);
+ }
+
+ private static List randomFields() {
+ return randomFieldAttributes(1, 5, false);
+ }
+
+ @Override
+ protected LookupJoinExec createTestInstance() {
+ return randomLookupJoinExec(0);
+ }
+
+ @Override
+ protected LookupJoinExec mutateInstance(LookupJoinExec instance) throws IOException {
+ PhysicalPlan child = instance.left();
+ PhysicalPlan lookup = instance.lookup();
+ List leftFields = randomFields();
+ List rightFields = randomFields();
+ List addedFields = randomFields();
+ switch (between(0, 4)) {
+ case 0 -> child = randomValueOtherThan(child, () -> randomChild(0));
+ case 1 -> lookup = randomValueOtherThan(lookup, () -> randomChild(0));
+ case 2 -> leftFields = randomValueOtherThan(leftFields, LookupJoinExecSerializationTests::randomFields);
+ case 3 -> rightFields = randomValueOtherThan(rightFields, LookupJoinExecSerializationTests::randomFields);
+ case 4 -> addedFields = randomValueOtherThan(addedFields, LookupJoinExecSerializationTests::randomFields);
+ default -> throw new UnsupportedOperationException();
+ }
+ return new LookupJoinExec(instance.source(), child, lookup, leftFields, rightFields, addedFields, null);
+ }
+
+ @Override
+ protected boolean alwaysEmptySource() {
+ return true;
+ }
+}
diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java
index 65f48ad06e299..c21aa9693989c 100644
--- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java
+++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java
@@ -514,7 +514,8 @@ public void accept(Page page) {
JoinTypes.LEFT,
List.of(UnresolvedAttributeTests.randomUnresolvedAttribute()),
List.of(UnresolvedAttributeTests.randomUnresolvedAttribute()),
- List.of(UnresolvedAttributeTests.randomUnresolvedAttribute())
+ List.of(UnresolvedAttributeTests.randomUnresolvedAttribute()),
+ null
);
}