Jelajahi Sumber

[ES|QL] Create Range in PushFiltersToSource for qualified pushable filters on the same field (#111437)

Fang Xing 1 tahun lalu
induk
melakukan
e7d99455c6

+ 0 - 6
docs/changelog/110548.yaml

@@ -1,6 +0,0 @@
-pr: 110548
-summary: "[ES|QL] Add `CombineBinaryComparisons` rule"
-area: ES|QL
-type: enhancement
-issues:
- - 108525

+ 5 - 0
docs/changelog/111437.yaml

@@ -0,0 +1,5 @@
+pr: 111437
+summary: "[ES|QL] Create `Range` in `PushFiltersToSource` for qualified pushable filters on the same field"
+area: ES|QL
+type: enhancement
+issues: []

+ 0 - 65
x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/ExpressionTranslators.java

@@ -7,12 +7,10 @@
 
 package org.elasticsearch.xpack.esql.core.planner;
 
-import org.elasticsearch.common.time.DateFormatter;
 import org.elasticsearch.xpack.esql.core.QlIllegalArgumentException;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
 import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute;
-import org.elasticsearch.xpack.esql.core.expression.predicate.Range;
 import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MatchQueryPredicate;
 import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MultiMatchQueryPredicate;
 import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.StringQueryPredicate;
@@ -32,31 +30,17 @@ import org.elasticsearch.xpack.esql.core.querydsl.query.MultiMatchQuery;
 import org.elasticsearch.xpack.esql.core.querydsl.query.NotQuery;
 import org.elasticsearch.xpack.esql.core.querydsl.query.Query;
 import org.elasticsearch.xpack.esql.core.querydsl.query.QueryStringQuery;
-import org.elasticsearch.xpack.esql.core.querydsl.query.RangeQuery;
 import org.elasticsearch.xpack.esql.core.querydsl.query.RegexQuery;
 import org.elasticsearch.xpack.esql.core.querydsl.query.WildcardQuery;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.util.Check;
 import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
 
-import java.time.OffsetTime;
-import java.time.ZonedDateTime;
-import java.time.temporal.TemporalAccessor;
 import java.util.Arrays;
 import java.util.List;
 
 public final class ExpressionTranslators {
 
-    public static final String DATE_FORMAT = "strict_date_optional_time_nanos";
-    public static final String TIME_FORMAT = "strict_hour_minute_second_fraction";
-
-    public static Object valueOf(Expression e) {
-        if (e.foldable()) {
-            return e.fold();
-        }
-        throw new QlIllegalArgumentException("Cannot determine value for {}", e);
-    }
-
     // TODO: see whether escaping is needed
     @SuppressWarnings("rawtypes")
     public static class Likes extends ExpressionTranslator<RegexMatch> {
@@ -193,55 +177,6 @@ public final class ExpressionTranslators {
         }
     }
 
-    public static class Ranges extends ExpressionTranslator<Range> {
-
-        @Override
-        protected Query asQuery(Range r, TranslatorHandler handler) {
-            return doTranslate(r, handler);
-        }
-
-        public static Query doTranslate(Range r, TranslatorHandler handler) {
-            return handler.wrapFunctionQuery(r, r.value(), () -> translate(r, handler));
-        }
-
-        private static RangeQuery translate(Range r, TranslatorHandler handler) {
-            Object lower = valueOf(r.lower());
-            Object upper = valueOf(r.upper());
-            String format = null;
-
-            // for a date constant comparison, we need to use a format for the date, to make sure that the format is the same
-            // no matter the timezone provided by the user
-            DateFormatter formatter = null;
-            if (lower instanceof ZonedDateTime || upper instanceof ZonedDateTime) {
-                formatter = DateFormatter.forPattern(DATE_FORMAT);
-            } else if (lower instanceof OffsetTime || upper instanceof OffsetTime) {
-                formatter = DateFormatter.forPattern(TIME_FORMAT);
-            }
-            if (formatter != null) {
-                // RangeQueryBuilder accepts an Object as its parameter, but it will call .toString() on the ZonedDateTime
-                // instance which can have a slightly different format depending on the ZoneId used to create the ZonedDateTime
-                // Since RangeQueryBuilder can handle date as String as well, we'll format it as String and provide the format.
-                if (lower instanceof ZonedDateTime || lower instanceof OffsetTime) {
-                    lower = formatter.format((TemporalAccessor) lower);
-                }
-                if (upper instanceof ZonedDateTime || upper instanceof OffsetTime) {
-                    upper = formatter.format((TemporalAccessor) upper);
-                }
-                format = formatter.pattern();
-            }
-            return new RangeQuery(
-                r.source(),
-                handler.nameOf(r.value()),
-                lower,
-                r.includeLower(),
-                upper,
-                r.includeUpper(),
-                format,
-                r.zoneId()
-            );
-        }
-    }
-
     public static Query or(Source source, Query left, Query right) {
         return boolQuery(source, left, right, false);
     }

+ 60 - 0
x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java

@@ -62,6 +62,7 @@ import static org.elasticsearch.test.MapMatcher.matchesMap;
 import static org.elasticsearch.xpack.esql.EsqlTestUtils.as;
 import static org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.Mode.ASYNC;
 import static org.elasticsearch.xpack.esql.qa.rest.RestEsqlTestCase.Mode.SYNC;
+import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateTimeToString;
 import static org.hamcrest.Matchers.any;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.either;
@@ -440,6 +441,65 @@ public abstract class RestEsqlTestCase extends ESRestTestCase {
         }
     }
 
+    // Test the Range created in PushFiltersToSource for qualified pushable filters on the same field
+    public void testInternalRange() throws IOException {
+        final int NUM_SINGLE_VALUE_ROWS = 100;
+        bulkLoadTestData(NUM_SINGLE_VALUE_ROWS);
+        bulkLoadTestData(10, NUM_SINGLE_VALUE_ROWS, false, RestEsqlTestCase::createDocumentWithMVs);
+        bulkLoadTestData(5, NUM_SINGLE_VALUE_ROWS + 10, false, RestEsqlTestCase::createDocumentWithNulls);
+
+        String upperBound = randomFrom(" < ", " <= ");
+        String lowerBound = randomFrom(" > ", " >= ");
+
+        String predicate = "{}" + upperBound + "{} and {}" + lowerBound + "{} and {} != {}";
+        int half = NUM_SINGLE_VALUE_ROWS / 2;
+        int halfPlusThree = half + 3;
+        List<String> predicates = List.of(
+            format(null, predicate, "integer", half, "integer", -1, "integer", half),
+            format(null, predicate, "short", half, "short", -1, "short", half),
+            format(null, predicate, "byte", half, "byte", -1, "byte", half),
+            format(null, predicate, "long", half, "long", -1, "long", half),
+            format(null, predicate, "double", half, "double", -1.0, "double", half),
+            format(null, predicate, "float", half, "float", -1.0, "float", half),
+            format(null, predicate, "half_float", half, "half_float", -1.0, "half_float", half),
+            format(null, predicate, "scaled_float", half, "scaled_float", -1.0, "scaled_float", half),
+            format(
+                null,
+                predicate,
+                "date",
+                "\"" + dateTimeToString(half) + "\"",
+                "date",
+                "\"1001-01-01\"",
+                "date",
+                "\"" + dateTimeToString(half) + "\""
+            ),
+            // keyword6-9 is greater than keyword53, [54,99] + [6, 9], 50 items in total
+            format(
+                null,
+                predicate,
+                "keyword",
+                "\"keyword999\"",
+                "keyword",
+                "\"keyword" + halfPlusThree + "\"",
+                "keyword",
+                "\"keyword" + halfPlusThree + "\""
+            ),
+            format(null, predicate, "ip", "\"127.0.0." + half + "\"", "ip", "\"126.0.0.0\"", "ip", "\"127.0.0." + half + "\""),
+            format(null, predicate, "version", "\"1.2." + half + "\"", "version", "\"1.2\"", "version", "\"1.2." + half + "\"")
+        );
+
+        for (String p : predicates) {
+            var query = requestObjectBuilder().query(format(null, "from {} | where {}", testIndexName(), p));
+            var result = runEsql(query, List.of(), NO_WARNINGS_REGEX, mode);
+            var values = as(result.get("values"), ArrayList.class);
+            assertThat(
+                format(null, "Comparison [{}] should return all rows with single values.", p),
+                values.size(),
+                is(NUM_SINGLE_VALUE_ROWS / 2)
+            );
+        }
+    }
+
     public void testWarningHeadersOnFailedConversions() throws IOException {
         int count = randomFrom(10, 40, 60);
         bulkLoadTestData(count);

+ 96 - 0
x-pack/plugin/esql/qa/testFixtures/src/main/resources/comparison.csv-spec

@@ -116,3 +116,99 @@ from employees
 emp_no:integer
 10001
 ;
+
+rangeInteger
+from employees
+| where emp_no > 10001 and emp_no < 10003
+| keep emp_no, first_name
+;
+
+emp_no:integer |first_name:keyword
+10002          |Bezalel
+;
+
+rangeLong
+from employees
+| where languages.long > 1 and languages.long < 3
+| keep emp_no, first_name
+| sort emp_no
+| limit 2
+;
+
+emp_no:integer |first_name:keyword
+10001          |Georgi
+10008          |Saniya
+;
+
+rangeDouble
+from employees
+| where height > 1.4 and height < 2.0
+| keep emp_no, first_name
+| sort emp_no
+| limit 2
+;
+
+emp_no:integer |first_name:keyword
+10003          |Parto
+10004          |Chirstian
+;
+
+rangeUnsignedLong
+from ul_logs
+| where bytes_out >= to_ul(4747671420480199905) and bytes_out <= to_ul(12749081495402663265)
+| keep id
+| sort id
+| limit 2
+;
+
+id:integer
+1
+3
+;
+
+
+rangeKeyword
+from employees
+| where first_name >= "A" and first_name <= "D"
+| keep emp_no, first_name
+| sort emp_no
+| limit 2
+;
+
+emp_no:integer |first_name:keyword
+10002          |Bezalel
+10004          |Chirstian
+;
+
+rangeVersion
+from apps
+| where version > "2" and version < "4"
+| keep id, version
+| sort id
+;
+
+id:integer |version:version
+2          |2.1
+3          |2.3.4
+4          |2.12.0
+;
+
+rangeDateTime
+from employees
+| where birth_date >= "1952-01-01" and birth_date <= "1952-12-31"
+| stats cnt = count(*)
+;
+
+cnt:long
+8
+;
+
+rangeMixed
+from employees
+| where birth_date >= "1952-01-01" and birth_date <= "1952-12-31" and hire_date >= "1980-01-01" and hire_date <= "1989-12-31"
+| stats cnt = count(*)
+;
+
+cnt:long
+5
+;

+ 1 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/spatial/SpatialRelatesUtils.java

@@ -28,7 +28,7 @@ import org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes;
 
 import java.io.IOException;
 
-import static org.elasticsearch.xpack.esql.core.planner.ExpressionTranslators.valueOf;
+import static org.elasticsearch.xpack.esql.core.expression.Foldables.valueOf;
 
 public class SpatialRelatesUtils {
 

+ 83 - 2
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/LocalPhysicalPlanOptimizer.java

@@ -30,6 +30,7 @@ import org.elasticsearch.xpack.esql.core.expression.NamedExpression;
 import org.elasticsearch.xpack.esql.core.expression.TypedAttribute;
 import org.elasticsearch.xpack.esql.core.expression.function.scalar.UnaryScalarFunction;
 import org.elasticsearch.xpack.esql.core.expression.predicate.Predicates;
+import org.elasticsearch.xpack.esql.core.expression.predicate.Range;
 import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MatchQueryPredicate;
 import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.StringQueryPredicate;
 import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And;
@@ -45,6 +46,7 @@ import org.elasticsearch.xpack.esql.core.rule.ParameterizedRuleExecutor;
 import org.elasticsearch.xpack.esql.core.rule.Rule;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.core.util.CollectionUtils;
 import org.elasticsearch.xpack.esql.core.util.Queries;
 import org.elasticsearch.xpack.esql.core.util.Queries.Clause;
 import org.elasticsearch.xpack.esql.core.util.StringUtils;
@@ -59,8 +61,12 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRe
 import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.StDistance;
 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.expression.predicate.operator.comparison.GreaterThan;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqual;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.In;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.InsensitiveBinaryComparison;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThan;
+import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual;
 import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals;
 import org.elasticsearch.xpack.esql.optimizer.PhysicalOptimizerRules.OptimizerRule;
 import org.elasticsearch.xpack.esql.plan.physical.AggregateExec;
@@ -252,8 +258,10 @@ public class LocalPhysicalPlanOptimizer extends ParameterizedRuleExecutor<Physic
                 for (Expression exp : splitAnd(filterExec.condition())) {
                     (canPushToSource(exp, x -> hasIdenticalDelegate(x, ctx.searchStats())) ? pushable : nonPushable).add(exp);
                 }
-                if (pushable.size() > 0) { // update the executable with pushable conditions
-                    Query queryDSL = TRANSLATOR_HANDLER.asQuery(Predicates.combineAnd(pushable));
+                // Combine GT, GTE, LT and LTE in pushable to Range if possible
+                List<Expression> newPushable = combineEligiblePushableToRange(pushable);
+                if (newPushable.size() > 0) { // update the executable with pushable conditions
+                    Query queryDSL = TRANSLATOR_HANDLER.asQuery(Predicates.combineAnd(newPushable));
                     QueryBuilder planQuery = queryDSL.asBuilder();
                     var query = Queries.combine(Clause.FILTER, asList(queryExec.query(), planQuery));
                     queryExec = new EsQueryExec(
@@ -277,6 +285,79 @@ public class LocalPhysicalPlanOptimizer extends ParameterizedRuleExecutor<Physic
             return plan;
         }
 
+        private static List<Expression> combineEligiblePushableToRange(List<Expression> pushable) {
+            List<EsqlBinaryComparison> bcs = new ArrayList<>();
+            List<Range> ranges = new ArrayList<>();
+            List<Expression> others = new ArrayList<>();
+            boolean changed = false;
+
+            pushable.forEach(e -> {
+                if (e instanceof GreaterThan || e instanceof GreaterThanOrEqual || e instanceof LessThan || e instanceof LessThanOrEqual) {
+                    if (((EsqlBinaryComparison) e).right().foldable()) {
+                        bcs.add((EsqlBinaryComparison) e);
+                    } else {
+                        others.add(e);
+                    }
+                } else {
+                    others.add(e);
+                }
+            });
+
+            for (int i = 0, step = 1; i < bcs.size() - 1; i += step, step = 1) {
+                BinaryComparison main = bcs.get(i);
+                for (int j = i + 1; j < bcs.size(); j++) {
+                    BinaryComparison other = bcs.get(j);
+                    if (main.left().semanticEquals(other.left())) {
+                        // >/>= AND </<=
+                        if ((main instanceof GreaterThan || main instanceof GreaterThanOrEqual)
+                            && (other instanceof LessThan || other instanceof LessThanOrEqual)) {
+                            bcs.remove(j);
+                            bcs.remove(i);
+
+                            ranges.add(
+                                new Range(
+                                    main.source(),
+                                    main.left(),
+                                    main.right(),
+                                    main instanceof GreaterThanOrEqual,
+                                    other.right(),
+                                    other instanceof LessThanOrEqual,
+                                    main.zoneId()
+                                )
+                            );
+
+                            changed = true;
+                            step = 0;
+                            break;
+                        }
+                        // </<= AND >/>=
+                        else if ((other instanceof GreaterThan || other instanceof GreaterThanOrEqual)
+                            && (main instanceof LessThan || main instanceof LessThanOrEqual)) {
+                                bcs.remove(j);
+                                bcs.remove(i);
+
+                                ranges.add(
+                                    new Range(
+                                        main.source(),
+                                        main.left(),
+                                        other.right(),
+                                        other instanceof GreaterThanOrEqual,
+                                        main.right(),
+                                        main instanceof LessThanOrEqual,
+                                        main.zoneId()
+                                    )
+                                );
+
+                                changed = true;
+                                step = 0;
+                                break;
+                            }
+                    }
+                }
+            }
+            return changed ? CollectionUtils.combine(others, bcs, ranges) : pushable;
+        }
+
         public static boolean canPushToSource(Expression exp, Predicate<FieldAttribute> hasIdenticalDelegate) {
             if (exp instanceof BinaryComparison bc) {
                 return isAttributePushable(bc.left(), bc, hasIdenticalDelegate) && bc.right().foldable();

+ 67 - 9
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsqlExpressionTranslators.java

@@ -18,6 +18,7 @@ import org.elasticsearch.xpack.esql.core.expression.Expressions;
 import org.elasticsearch.xpack.esql.core.expression.FieldAttribute;
 import org.elasticsearch.xpack.esql.core.expression.TypedAttribute;
 import org.elasticsearch.xpack.esql.core.expression.function.scalar.ScalarFunction;
+import org.elasticsearch.xpack.esql.core.expression.predicate.Range;
 import org.elasticsearch.xpack.esql.core.expression.predicate.operator.comparison.BinaryComparison;
 import org.elasticsearch.xpack.esql.core.planner.ExpressionTranslator;
 import org.elasticsearch.xpack.esql.core.planner.ExpressionTranslators;
@@ -55,6 +56,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
+import static org.elasticsearch.xpack.esql.core.expression.Foldables.valueOf;
 import static org.elasticsearch.xpack.esql.core.planner.ExpressionTranslators.or;
 import static org.elasticsearch.xpack.esql.core.type.DataType.IP;
 import static org.elasticsearch.xpack.esql.core.type.DataType.UNSIGNED_LONG;
@@ -67,14 +69,13 @@ import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.ipToString
 import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.versionToString;
 
 public final class EsqlExpressionTranslators {
+
     public static final List<ExpressionTranslator<?>> QUERY_TRANSLATORS = List.of(
         new EqualsIgnoreCaseTranslator(),
         new BinaryComparisons(),
         new SpatialRelatesTranslator(),
         new InComparisons(),
-        // Ranges is redundant until we start combining binary comparisons (see CombineBinaryComparisons in ql's OptimizerRules)
-        // or introduce a BETWEEN keyword.
-        new ExpressionTranslators.Ranges(),
+        new Ranges(), // Create Range in PushFiltersToSource for qualified pushable filters on the same field.
         new ExpressionTranslators.BinaryLogic(),
         new ExpressionTranslators.IsNulls(),
         new ExpressionTranslators.IsNotNulls(),
@@ -124,7 +125,7 @@ public final class EsqlExpressionTranslators {
         static Query translate(InsensitiveEquals bc) {
             TypedAttribute attribute = checkIsPushableAttribute(bc.left());
             Source source = bc.source();
-            BytesRef value = BytesRefs.toBytesRef(ExpressionTranslators.valueOf(bc.right()));
+            BytesRef value = BytesRefs.toBytesRef(valueOf(bc.right()));
             String name = pushableAttributeName(attribute);
             return new TermQuery(source, name, value.utf8ToString(), true);
         }
@@ -249,7 +250,7 @@ public final class EsqlExpressionTranslators {
                 return null;
             }
             Source source = bc.source();
-            Object value = ExpressionTranslators.valueOf(bc.right());
+            Object value = valueOf(bc.right());
 
             // Comparisons with multi-values always return null in ESQL.
             if (value instanceof List<?>) {
@@ -453,12 +454,69 @@ public final class EsqlExpressionTranslators {
 
             return queries.stream().reduce((q1, q2) -> or(in.source(), q1, q2)).get();
         }
+    }
+
+    public static class Ranges extends ExpressionTranslator<Range> {
+
+        @Override
+        protected Query asQuery(Range r, TranslatorHandler handler) {
+            return doTranslate(r, handler);
+        }
+
+        public static Query doTranslate(Range r, TranslatorHandler handler) {
+            return handler.wrapFunctionQuery(r, r.value(), () -> translate(r, handler));
+        }
+
+        private static RangeQuery translate(Range r, TranslatorHandler handler) {
+            Object lower = valueOf(r.lower());
+            Object upper = valueOf(r.upper());
+            String format = null;
+
+            DataType dataType = r.value().dataType();
+            if (DataType.isDateTime(dataType) && DataType.isDateTime(r.lower().dataType()) && DataType.isDateTime(r.upper().dataType())) {
+                lower = dateTimeToString((Long) lower);
+                upper = dateTimeToString((Long) upper);
+                format = DEFAULT_DATE_TIME_FORMATTER.pattern();
+            }
 
-        public static Object valueOf(Expression e) {
-            if (e.foldable()) {
-                return e.fold();
+            if (dataType == IP) {
+                if (lower instanceof BytesRef bytesRef) {
+                    lower = ipToString(bytesRef);
+                }
+                if (upper instanceof BytesRef bytesRef) {
+                    upper = ipToString(bytesRef);
+                }
+            } else if (dataType == VERSION) {
+                // VersionStringFieldMapper#indexedValueForSearch() only accepts as input String or BytesRef with the String (i.e. not
+                // encoded) representation of the version as it'll do the encoding itself.
+                if (lower instanceof BytesRef bytesRef) {
+                    lower = versionToString(bytesRef);
+                } else if (lower instanceof Version version) {
+                    lower = versionToString(version);
+                }
+                if (upper instanceof BytesRef bytesRef) {
+                    upper = versionToString(bytesRef);
+                } else if (upper instanceof Version version) {
+                    upper = versionToString(version);
+                }
+            } else if (dataType == UNSIGNED_LONG) {
+                if (lower instanceof Long ul) {
+                    lower = unsignedLongAsNumber(ul);
+                }
+                if (upper instanceof Long ul) {
+                    upper = unsignedLongAsNumber(ul);
+                }
             }
-            throw new QlIllegalArgumentException("Cannot determine value for {}", e);
+            return new RangeQuery(
+                r.source(),
+                handler.nameOf(r.value()),
+                lower,
+                r.includeLower(),
+                upper,
+                r.includeUpper(),
+                format,
+                r.zoneId()
+            );
         }
     }
 }

+ 85 - 24
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/QueryTranslatorTests.java

@@ -113,55 +113,108 @@ public class QueryTranslatorTests extends ESTestCase {
     }
 
     public void testRanges() {
-        // Note: Currently binary comparisons are not combined into range queries, so we get bool queries with multiple
-        // one-sided ranges for now.
-
-        // Once we combine binary comparisons, this query should be trivial.
+        // ORs
         assertQueryTranslation("""
             FROM test | WHERE 10 < integer OR integer < 12""", matchesRegex("""
             .*should.*""" + """
             esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"gt":10,.*""" + """
             esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"lt":12.*"""));
 
+        assertQueryTranslation("""
+            FROM test | WHERE 10 <= integer OR integer < 12""", matchesRegex("""
+            .*should.*""" + """
+            esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"gte":10,.*""" + """
+            esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"lt":12.*"""));
+
+        assertQueryTranslation("""
+            FROM test | WHERE 10 < integer OR integer <= 12""", matchesRegex("""
+            .*should.*""" + """
+            esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"gt":10,.*""" + """
+            esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"lte":12.*"""));
+
+        assertQueryTranslation("""
+            FROM test | WHERE 10 <= integer OR integer <= 12""", matchesRegex("""
+            .*should.*""" + """
+            esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"gte":10,.*""" + """
+            esql_single_value":\\{"field":"integer".*"range":\\{"integer":\\{"lte":12.*"""));
+
+        // ANDs
         assertQueryTranslation("""
             FROM test | WHERE 10 < integer AND integer < 12""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"integer\"""" + """
-            .*"range":\\{"integer":\\{"gt":10,.*"range":\\{"integer":\\{"lt":12.*"""));
+            \\{"esql_single_value":\\{"field":"integer","next":\\{"range":\\{"integer":\\{"gt":10,"lt":12.*"""));
 
         assertQueryTranslation("""
             FROM test | WHERE 10 <= integer AND integer <= 12""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"integer\"""" + """
-            .*"range":\\{"integer":\\{"gte":10,.*"range":\\{"integer":\\{"lte":12.*"""));
+            \\{"esql_single_value":\\{"field":"integer","next":\\{"range":\\{"integer":\\{"gte":10,"lte":12.*"""));
+
+        assertQueryTranslation("""
+            FROM test | WHERE 10 <= integer AND integer < 12""", matchesRegex("""
+            \\{"esql_single_value":\\{"field":"integer","next":\\{"range":\\{"integer":\\{"gte":10,"lt":12.*"""));
 
         assertQueryTranslation("""
             FROM test | WHERE 10.9 < double AND double < 12.1""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"double\"""" + """
-            .*"range":\\{"double":\\{"gt":10.9,.*"range":\\{"double":\\{"lt":12.1.*"""));
+            \\{"esql_single_value":\\{"field":"double","next":\\{"range":\\{"double":\\{"gt":10.9,"lt":12.1.*"""));
 
         assertQueryTranslation("""
             FROM test | WHERE 10.9 <= double AND double <= 12.1""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"double\"""" + """
-            .*"range":\\{"double":\\{"gte":10.9,.*"range":\\{"double":\\{"lte":12.1.*"""));
+            \\{"esql_single_value":\\{"field":"double","next":\\{"range":\\{"double":\\{"gte":10.9,"lte":12.1.*"""));
 
         assertQueryTranslation("""
-            FROM test | WHERE "2007-12-03T10:15:30+01:00" < date AND date < "2024-01-01T10:15:30+01:00\"""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"date\"""" + """
-            .*"range":\\{"date":\\{"gt":\"2007-12-03T09:15:30.000Z\",.*"range":\\{"date":\\{"lt":\"2024-01-01T09:15:30.000Z\".*"""));
+            FROM test | WHERE 10.9 < double AND double <= 12.1""", matchesRegex("""
+            \\{"esql_single_value":\\{"field":"double","next":\\{"range":\\{"double":\\{"gt":10.9,"lte":12.1.*"""));
 
         assertQueryTranslation("""
-            FROM test | WHERE "2007-12-03T10:15:30+01:00" <= date AND date <= "2024-01-01T10:15:30+01:00\"""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"date\"""" + """
-            .*"range":\\{"date":\\{"gte":\"2007-12-03T09:15:30.000Z\",.*"range":\\{"date":\\{"lte":\"2024-01-01T09:15:30.000Z\".*"""));
+            FROM test | WHERE 2147483648::unsigned_long < unsigned_long AND unsigned_long < 2147483650::unsigned_long""", matchesRegex("""
+            \\{"esql_single_value":\\{"field":"unsigned_long".*\\{"range":\\{"unsigned_long":\\{"gt":2147483648,"lt":2147483650.*"""));
 
         assertQueryTranslation("""
-            FROM test | WHERE 2147483648::unsigned_long < unsigned_long AND unsigned_long < 2147483650::unsigned_long""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"unsigned_long\"""" + """
-            .*"range":\\{"unsigned_long":\\{"gt":2147483648,.*"range":\\{"unsigned_long":\\{"lt":2147483650,.*"""));
+            FROM test | WHERE 2147483648::unsigned_long <= unsigned_long AND unsigned_long <= 2147483650::unsigned_long""", matchesRegex("""
+            \\{"esql_single_value":\\{"field":"unsigned_long".*\\{"range":\\{"unsigned_long":\\{"gte":2147483648,"lte":2147483650.*"""));
 
         assertQueryTranslation("""
             FROM test | WHERE 2147483648::unsigned_long <= unsigned_long AND unsigned_long <= 2147483650::unsigned_long""", matchesRegex("""
-            .*must.*esql_single_value":\\{"field":"unsigned_long\"""" + """
-            .*"range":\\{"unsigned_long":\\{"gte":2147483648,.*"range":\\{"unsigned_long":\\{"lte":2147483650,.*"""));
+            \\{"esql_single_value":\\{"field":"unsigned_long".*\\{"range":\\{"unsigned_long":\\{"gte":2147483648,"lte":2147483650.*"""));
+
+        // mixed ANDs and NotEquals
+        assertQueryTranslation("""
+            FROM test | WHERE 10 < integer AND integer < 12 AND integer > 0 AND integer != 5""", matchesRegex("""
+            .*bool.*must.*""" + """
+            esql_single_value":\\{"field":"integer","next":\\{"bool":""" + """
+            .*must_not.*\\[\\{"term":\\{"integer":\\{"value":5.*""" + """
+            esql_single_value":\\{"field":"integer","next":\\{"range":\\{"integer":\\{"gt":10,"lt":12.*"""));
+
+        // multiple Ranges
+        assertQueryTranslation("""
+            FROM test | WHERE 10 < integer AND double < 1.0 AND integer < 12 AND double > -1.0""", matchesRegex("""
+            .*bool.*must.*""" + """
+            esql_single_value":\\{"field":"integer","next":\\{"range":\\{"integer":\\{"gt":10,"lt":12.*""" + """
+            esql_single_value":\\{"field":"double","next":\\{"range":\\{"double":\\{"gt":-1.0,"lt":1.0.*"""));
+
+        assertQueryTranslation("""
+            FROM test | WHERE "2007-12-03T10:15:30Z" <= date AND date <= "2024-01-01T10:15:30\"""", containsString("""
+            "esql_single_value":{"field":"date","next":{"range":{"date":{"gte":"2007-12-03T10:15:30.000Z","lte":"2024-01-01T10:15:30.000Z",\
+            "time_zone":"Z","format":"strict_date_optional_time","boost":1.0}}}"""));
+
+        assertQueryTranslation("""
+            FROM test | WHERE "2007-12-03T10:15:30" <= date AND date <= "2024-01-01T10:15:30Z\"""", containsString("""
+            "esql_single_value":{"field":"date","next":{"range":{"date":{"gte":"2007-12-03T10:15:30.000Z","lte":"2024-01-01T10:15:30.000Z",\
+            "time_zone":"Z","format":"strict_date_optional_time","boost":1.0}}}"""));
+
+        // various timezones
+        assertQueryTranslation("""
+            FROM test | WHERE "2007-12-03T10:15:30+01:00" < date AND date < "2024-01-01T10:15:30+01:00\"""", containsString("""
+            "esql_single_value":{"field":"date","next":{"range":{"date":{"gt":"2007-12-03T09:15:30.000Z","lt":"2024-01-01T09:15:30.000Z",\
+            "time_zone":"Z","format":"strict_date_optional_time","boost":1.0}}}"""));
+
+        assertQueryTranslation("""
+            FROM test | WHERE "2007-12-03T10:15:30-01:00" <= date AND date <= "2024-01-01T10:15:30+01:00\"""", containsString("""
+            "esql_single_value":{"field":"date","next":{"range":{"date":{"gte":"2007-12-03T11:15:30.000Z","lte":"2024-01-01T09:15:30.000Z",\
+            "time_zone":"Z","format":"strict_date_optional_time","boost":1.0}}}"""));
+
+        assertQueryTranslation("""
+            FROM test | WHERE "2007-12-03T10:15:30" <= date AND date <= "2024-01-01T10:15:30+01:00\"""", containsString("""
+            "esql_single_value":{"field":"date","next":{"range":{"date":{"gte":"2007-12-03T10:15:30.000Z","lte":"2024-01-01T09:15:30.000Z",\
+            "time_zone":"Z","format":"strict_date_optional_time","boost":1.0}}}"""));
     }
 
     public void testIPs() {
@@ -175,7 +228,15 @@ public class QueryTranslatorTests extends ESTestCase {
             esql_single_value":\\{"field":"ip0".*"terms":\\{"ip0":\\["127.0.0.3".*""" + """
             esql_single_value":\\{"field":"ip1".*"terms":\\{"ip1":\\["fe80::cae2:65ff:fece:fec0".*"""));
 
-        // Combine Equals, In and CIDRMatch on IP type
+        // ANDs
+        assertQueryTranslationIPs("""
+            FROM hosts | WHERE ip1 >= "127.0.0.1" AND ip1 <= "128.0.0.1" \
+            AND ip0 > "127.0.0.1" AND  ip0 < "128.0.0.1\"""", matchesRegex("""
+            .*bool.*must.*""" + """
+            esql_single_value":\\{"field":"ip1".*"range":\\{"ip1":\\{"gte":"127.0.0.1","lte":"128.0.0.1".*""" + """
+            esql_single_value":\\{"field":"ip0".*"range":\\{"ip0":\\{"gt":"127.0.0.1","lt":"128.0.0.1".*"""));
+
+        // ORs - Combine Equals, In and CIDRMatch on IP type
         assertQueryTranslationIPs("""
             FROM hosts | WHERE host == "alpha" OR host == "gamma" OR CIDR_MATCH(ip1, "127.0.0.2/32") OR CIDR_MATCH(ip1, "127.0.0.3/32") \
             OR card IN ("eth0", "eth1") OR card == "lo0" OR CIDR_MATCH(ip0, "127.0.0.1") OR \

+ 42 - 0
x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/esql/10_basic.yml

@@ -109,6 +109,30 @@ setup:
           - { "data": 1, "count": 44, "data_d": 1, "count_d": 44, "time": 1674835275225, "color": "green" }
           - { "index": { } }
           - { "data": 2, "count": 46, "data_d": 2, "count_d": 46, "time": 1674835275226, "color": "red" }
+  - do:
+      indices.create:
+        index:  test_range
+        body:
+          settings:
+            number_of_shards: 5
+          mappings:
+            properties:
+              date_1:
+                type: date
+                format: "dd-MM-yyyy-MM-dd HH:mm Z"
+              date_2:
+                type: date
+  - do:
+      bulk:
+        index: "test_range"
+        refresh: true
+        body:
+          - { "index": { } }
+          - { "date_1": "12-01-1981-01-12 15:00 +0000", "date_2": "2000" }
+          - { "index": { } }
+          - { "date_1": "31-12-1999-12-31 23:59 +0300" }
+          - { "index": { } }
+          - { "date_1": "31-12-1999-12-31 23:59 -0300" }
 
 ---
 "Test From":
@@ -380,3 +404,21 @@ version is not allowed:
         body:
           query: 'from test'
           version: cat
+
+---
+"Test Internal Range":
+  - do:
+      allowed_warnings_regex:
+        - "No limit defined, adding default limit of \\[.*\\]"
+      esql.query:
+        body:
+          query: 'from test_range | where date_1 > "1990" and date_1 < "2000" or date_2 >= "2000" | sort date_1'
+
+  - length: {columns: 2}
+  - match: {columns.0.name: "date_1"}
+  - match: {columns.0.type: "date"}
+  - match: {columns.1.name: "date_2"}
+  - match: {columns.1.type: "date"}
+  - length: {values: 2}
+  - match: {values.0: ["1981-01-12T15:00:00.000Z","2000-01-01T00:00:00.000Z"]}
+  - match: {values.1: ["1999-12-31T20:59:00.000Z", null]}