1
0
Эх сурвалжийг харах

SQL: Fix ORDER BY YEAR() function (#51562)

Previously, if YEAR() was used as and ORDER BY argument without being
wrapped with another scalar (e.g. YEAR(birth_date) + 10), no script
ordering was used but instead the underlying field (e.g. birth_date)
was used instead as a performance optimisation. This works correctly if
YEAR() is the only ORDER BY arg but if further args are used as tie
breakers for the ordering wrong results are produced. This is because
2 rows with the different birth_date but on the same year are not tied
as the underlying ordering is on birth_date and not on the
YEAR(birth_date), and the following ORDER BY args are ignored.

Remove this optimisation for YEAR() to avoid incorrect results in
such cases.

As a consequence another bug is revealed: scalar functions on top
of nested fields produce scripted sorting/filtering which is not yet
supported. In such cases no error was thrown but instead all values for
such nested fields were null and were passed to the script implementing
the sorting/filtering, producing incorrect results.

Detect such cases and throw a validation exception.

Fixes: #51224
Marios Trivyzas 5 жил өмнө
parent
commit
f41efd6753

+ 28 - 0
docs/reference/sql/limitations.asciidoc

@@ -31,6 +31,34 @@ For example:
 SELECT dep.dep_name.keyword FROM test_emp GROUP BY languages;
 --------------------------------------------------
 
+[float]
+=== Scalar functions on nested fields are not allowed in `WHERE` and `ORDER BY` clauses
+{es-sql} doesn't support the usage of scalar functions on top of nested fields in `WHERE`
+and `ORDER BY` clauses with the exception of comparison and logical operators.
+
+For example:
+
+[source, sql]
+--------------------------------------------------
+SELECT * FROM test_emp WHERE LENGTH(dep.dep_name.keyword) > 5;
+--------------------------------------------------
+
+and
+
+[source, sql]
+--------------------------------------------------
+SELECT * FROM test_emp ORDER BY YEAR(dep.start_date);
+--------------------------------------------------
+
+are not supported but:
+
+[source, sql]
+--------------------------------------------------
+SELECT * FROM test_emp WHERE dep.start_date >= CAST('2020-01-01' AS DATE) OR dep.dep_end_date IS NULL;
+--------------------------------------------------
+
+is supported.
+
 [float]
 === Multi-nested fields
 

+ 1 - 10
x-pack/plugin/ql/src/main/java/org/elasticsearch/xpack/ql/expression/function/scalar/ScalarFunction.java

@@ -39,17 +39,9 @@ public abstract class ScalarFunction extends Function {
         super(source, fields);
     }
 
-    // used if the function is monotonic and thus does not have to be computed for ordering purposes
-    // null means the script needs to be used; expression means the field/expression to be used instead
-    public Expression orderBy() {
-        return null;
-    }
-
-
     //
     // Script generation
     //
-
     public ScriptTemplate asScript(Expression exp) {
         if (exp.foldable()) {
             return scriptWithFoldable(exp);
@@ -73,7 +65,6 @@ public abstract class ScalarFunction extends Function {
         throw new QlIllegalArgumentException("Cannot evaluate script for expression {}", exp);
     }
 
-
     protected ScriptTemplate scriptWithFoldable(Expression foldable) {
         Object fold = foldable.fold();
 
@@ -144,4 +135,4 @@ public abstract class ScalarFunction extends Function {
     protected String formatTemplate(String template) {
         return Scripts.formatTemplate(template);
     }
-}
+}

+ 6 - 0
x-pack/plugin/sql/qa/src/main/resources/datetime.sql-spec

@@ -124,6 +124,12 @@ SELECT first_name FROM test_emp ORDER BY NOW(), first_name NULLS LAST LIMIT 5;
 groupByCurrentTimestamp
 SELECT MAX(salary) AS max FROM test_emp GROUP BY NOW();
 
+//
+// ORDER BY
+//
+orderByYear
+SELECT YEAR(birth_date) as year, emp_no FROM test_emp ORDER BY year NULLS FIRST, emp_no ASC;
+
 // ES will consider a TIME in UTC, if no other indication is given and H2 doesn't consider the timezone for times, so no TZSync'ing needed.
 hourFromStringTime
 SELECT HOUR(CAST('18:09:03' AS TIME)) AS result;

+ 1 - 1
x-pack/plugin/sql/qa/src/main/resources/docs/docs.csv-spec

@@ -939,7 +939,7 @@ SELECT -2 * INTERVAL '3' YEARS AS result;
 
 ///////////////////////////////
 //
-// Order by
+// Order By
 //
 ///////////////////////////////
 

+ 6 - 18
x-pack/plugin/sql/qa/src/main/resources/nested.csv-spec

@@ -102,27 +102,15 @@ Mayuko     | 12
 ;
 
 selectWithScalarOnNested
-SELECT first_name f, last_name l, YEAR(dep.from_date) start FROM test_emp WHERE dep.dep_name = 'Production' AND languages > 1 ORDER BY start LIMIT 5;
+SELECT first_name f, last_name l, YEAR(dep.from_date) start FROM test_emp WHERE dep.dep_name = 'Production' AND languages > 1 ORDER BY dep.from_date LIMIT 5;
 
 f:s                  | l:s           | start:i
 
-Sreekrishna          |Servieres      |1985           
-Zhongwei             |Rosen          |1986           
-Chirstian            |Koblick        |1986           
-null                 |Chappelet      |1988           
-Zvonko               |Nyanchama      |1989     
-;
-
-selectWithScalarOnNestedWithoutProjection
-SELECT first_name f, last_name l FROM test_emp WHERE dep.dep_name = 'Production' AND languages > 1 ORDER BY YEAR(dep.from_date) LIMIT 5;
-
-f:s                  | l:s
-
-Sreekrishna          |Servieres      
-Zhongwei             |Rosen          
-Chirstian            |Koblick        
-null                 |Chappelet      
-Zvonko               |Nyanchama  
+Sreekrishna          |Servieres      |1985
+Zhongwei             |Rosen          |1986
+Chirstian            |Koblick        |1986
+null                 |Chappelet      |1988
+Zvonko               |Nyanchama      |1989
 ;
 
 //

+ 44 - 11
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Verifier.java

@@ -23,6 +23,9 @@ import org.elasticsearch.xpack.ql.expression.function.aggregate.AggregateFunctio
 import org.elasticsearch.xpack.ql.expression.function.grouping.GroupingFunction;
 import org.elasticsearch.xpack.ql.expression.function.scalar.ScalarFunction;
 import org.elasticsearch.xpack.ql.expression.predicate.fulltext.FullTextPredicate;
+import org.elasticsearch.xpack.ql.expression.predicate.logical.BinaryLogic;
+import org.elasticsearch.xpack.ql.expression.predicate.logical.Not;
+import org.elasticsearch.xpack.ql.expression.predicate.operator.comparison.BinaryComparison;
 import org.elasticsearch.xpack.ql.plan.logical.Aggregate;
 import org.elasticsearch.xpack.ql.plan.logical.Filter;
 import org.elasticsearch.xpack.ql.plan.logical.Limit;
@@ -40,6 +43,8 @@ import org.elasticsearch.xpack.sql.expression.function.Score;
 import org.elasticsearch.xpack.sql.expression.function.aggregate.Max;
 import org.elasticsearch.xpack.sql.expression.function.aggregate.Min;
 import org.elasticsearch.xpack.sql.expression.function.aggregate.TopHits;
+import org.elasticsearch.xpack.sql.expression.predicate.nulls.IsNotNull;
+import org.elasticsearch.xpack.sql.expression.predicate.nulls.IsNull;
 import org.elasticsearch.xpack.sql.plan.logical.Distinct;
 import org.elasticsearch.xpack.sql.plan.logical.LocalRelation;
 import org.elasticsearch.xpack.sql.plan.logical.Pivot;
@@ -255,7 +260,7 @@ public final class Verifier {
                 }
 
                 checkForScoreInsideFunctions(p, localFailures);
-                checkNestedUsedInGroupByOrHaving(p, localFailures);
+                checkNestedUsedInGroupByOrHavingOrWhereOrOrderBy(p, localFailures, attributeRefs);
                 checkForGeoFunctionsOnDocValues(p, localFailures);
                 checkPivot(p, localFailures, attributeRefs);
 
@@ -735,17 +740,21 @@ public final class Verifier {
                 Function.class));
     }
 
-    private static void checkNestedUsedInGroupByOrHaving(LogicalPlan p, Set<Failure> localFailures) {
+    private static void checkNestedUsedInGroupByOrHavingOrWhereOrOrderBy(LogicalPlan p, Set<Failure> localFailures,
+                                                                         AttributeMap<Expression> attributeRefs) {
         List<FieldAttribute> nested = new ArrayList<>();
-        Consumer<FieldAttribute> match = fa -> {
+        Consumer<FieldAttribute> matchNested = fa -> {
             if (fa.isNested()) {
                 nested.add(fa);
             }
         };
+        Consumer<Expression> checkForNested = e ->
+                attributeRefs.getOrDefault(e, e).forEachUp(matchNested, FieldAttribute.class);
+        Consumer<ScalarFunction> checkForNestedInFunction =  f -> f.arguments().forEach(
+                arg -> arg.forEachUp(matchNested, FieldAttribute.class));
 
         // nested fields shouldn't be used in aggregates or having (yet)
-        p.forEachDown(a -> a.groupings().forEach(agg -> agg.forEachUp(match, FieldAttribute.class)), Aggregate.class);
-
+        p.forEachDown(a -> a.groupings().forEach(agg -> agg.forEachUp(checkForNested)), Aggregate.class);
         if (!nested.isEmpty()) {
             localFailures.add(
                     fail(nested.get(0), "Grouping isn't (yet) compatible with nested fields " + new AttributeSet(nested).names()));
@@ -753,15 +762,39 @@ public final class Verifier {
         }
 
         // check in having
-        p.forEachDown(f -> {
-            if (f.child() instanceof Aggregate) {
-                f.condition().forEachUp(match, FieldAttribute.class);
-            }
-        }, Filter.class);
-
+        p.forEachDown(f -> f.forEachDown(a -> f.condition().forEachUp(checkForNested), Aggregate.class), Filter.class);
         if (!nested.isEmpty()) {
             localFailures.add(
                     fail(nested.get(0), "HAVING isn't (yet) compatible with nested fields " + new AttributeSet(nested).names()));
+            nested.clear();
+        }
+
+        // check in where (scalars not allowed)
+        p.forEachDown(f -> f.condition().forEachUp(e ->
+                attributeRefs.getOrDefault(e, e).forEachUp(sf -> {
+                    if (sf instanceof BinaryComparison == false &&
+                            sf instanceof IsNull == false &&
+                            sf instanceof IsNotNull == false &&
+                            sf instanceof Not == false &&
+                            sf instanceof BinaryLogic== false) {
+                        checkForNestedInFunction.accept(sf);
+                    }}, ScalarFunction.class)
+        ), Filter.class);
+        if (!nested.isEmpty()) {
+            localFailures.add(
+                    fail(nested.get(0), "WHERE isn't (yet) compatible with scalar functions on nested fields " +
+                            new AttributeSet(nested).names()));
+            nested.clear();
+        }
+
+        // check in order by (scalars not allowed)
+        p.forEachDown(ob -> ob.order().forEach(o -> o.forEachUp(e ->
+                attributeRefs.getOrDefault(e, e).forEachUp(checkForNestedInFunction, ScalarFunction.class)
+        )), OrderBy.class);
+        if (!nested.isEmpty()) {
+            localFailures.add(
+                    fail(nested.get(0), "ORDER BY isn't (yet) compatible with scalar functions on nested fields " +
+                            new AttributeSet(nested).names()));
         }
     }
 

+ 0 - 5
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/Year.java

@@ -39,11 +39,6 @@ public class Year extends DateTimeHistogramFunction {
         return "year";
     }
 
-    @Override
-    public Expression orderBy() {
-        return field();
-    }
-
     @Override
     public String calendarInterval() {
         return YEAR_INTERVAL;

+ 2 - 15
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java

@@ -695,21 +695,8 @@ class QueryFolder extends RuleExecutor<PhysicalPlan> {
                         // scalar functions typically require script ordering
                         if (orderExpression instanceof ScalarFunction) {
                             ScalarFunction sf = (ScalarFunction) orderExpression;
-                            // is there an expression to order by?
-                            if (sf.orderBy() != null) {
-                                Expression orderBy = sf.orderBy();
-                                if (orderBy instanceof NamedExpression) {
-                                    orderBy = qContainer.aliases().getOrDefault(orderBy, orderBy);
-                                    qContainer = qContainer
-                                            .addSort(new AttributeSort(((NamedExpression) orderBy).toAttribute(), direction, missing));
-                                } else if (orderBy.foldable() == false) {
-                                    // ignore constant
-                                    throw new PlanningException("does not know how to order by expression {}", orderBy);
-                                }
-                            } else {
-                                // nope, use scripted sorting
-                                qContainer = qContainer.addSort(new ScriptSort(sf.asScript(), direction, missing));
-                            }
+                            // nope, use scripted sorting
+                            qContainer = qContainer.addSort(new ScriptSort(sf.asScript(), direction, missing));
                         }
                         // score
                         else if (orderExpression instanceof Score) {

+ 35 - 0
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java

@@ -470,11 +470,46 @@ public class VerifierErrorMessagesTests extends ESTestCase {
     public void testGroupByOnNested() {
         assertEquals("1:38: Grouping isn't (yet) compatible with nested fields [dep.dep_id]",
                 error("SELECT dep.dep_id FROM test GROUP BY dep.dep_id"));
+        assertEquals("1:8: Grouping isn't (yet) compatible with nested fields [dep.dep_id]",
+                error("SELECT dep.dep_id AS a FROM test GROUP BY a"));
+        assertEquals("1:8: Grouping isn't (yet) compatible with nested fields [dep.dep_id]",
+                error("SELECT dep.dep_id AS a FROM test GROUP BY 1"));
+        assertEquals("1:8: Grouping isn't (yet) compatible with nested fields [dep.dep_id, dep.start_date]",
+                error("SELECT dep.dep_id AS a, dep.start_date AS b FROM test GROUP BY 1, 2"));
+        assertEquals("1:8: Grouping isn't (yet) compatible with nested fields [dep.dep_id, dep.start_date]",
+                error("SELECT dep.dep_id AS a, dep.start_date AS b FROM test GROUP BY a, b"));
     }
 
     public void testHavingOnNested() {
         assertEquals("1:51: HAVING isn't (yet) compatible with nested fields [dep.start_date]",
                 error("SELECT int FROM test GROUP BY int HAVING AVG(YEAR(dep.start_date)) > 1980"));
+        assertEquals("1:22: HAVING isn't (yet) compatible with nested fields [dep.start_date]",
+                error("SELECT int, AVG(YEAR(dep.start_date)) AS average FROM test GROUP BY int HAVING average > 1980"));
+        assertEquals("1:22: HAVING isn't (yet) compatible with nested fields [dep.start_date, dep.end_date]",
+                error("SELECT int, AVG(YEAR(dep.start_date)) AS a, MAX(MONTH(dep.end_date)) AS b " +
+                      "FROM test GROUP BY int " +
+                      "HAVING a > 1980 AND b < 10"));
+    }
+
+    public void testWhereOnNested() {
+        assertEquals("1:33: WHERE isn't (yet) compatible with scalar functions on nested fields [dep.start_date]",
+                error("SELECT int FROM test WHERE YEAR(dep.start_date) + 10 > 0"));
+        assertEquals("1:13: WHERE isn't (yet) compatible with scalar functions on nested fields [dep.start_date]",
+                error("SELECT YEAR(dep.start_date) + 10 AS a FROM test WHERE int > 10 AND (int < 3 OR NOT (a > 5))"));
+        accept("SELECT int FROM test WHERE dep.start_date > '2020-01-30'::date AND (int > 10 OR dep.end_date IS NULL)");
+        accept("SELECT int FROM test WHERE dep.start_date > '2020-01-30'::date AND (int > 10 OR dep.end_date IS NULL) " +
+               "OR NOT(dep.start_date >= '2020-01-01')");
+    }
+
+    public void testOrderByOnNested() {
+        assertEquals("1:36: ORDER BY isn't (yet) compatible with scalar functions on nested fields [dep.start_date]",
+                error("SELECT int FROM test ORDER BY YEAR(dep.start_date) + 10"));
+        assertEquals("1:13: ORDER BY isn't (yet) compatible with scalar functions on nested fields [dep.start_date]",
+                error("SELECT YEAR(dep.start_date) + 10  FROM test ORDER BY 1"));
+        assertEquals("1:13: ORDER BY isn't (yet) compatible with scalar functions on nested fields " +
+                        "[dep.start_date, dep.end_date]",
+                error("SELECT YEAR(dep.start_date) + 10 AS a, MONTH(dep.end_date) - 10 as b FROM test ORDER BY 1, 2"));
+        accept("SELECT int FROM test ORDER BY dep.start_date, dep.end_date");
     }
 
     public void testGroupByScalarFunctionWithAggOnTarget() {

+ 14 - 0
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java

@@ -1025,6 +1025,20 @@ public class QueryTranslatorTests extends ESTestCase {
                         "\"calendar_interval\":\"1y\",\"time_zone\":\"Z\"}}}]}}}"));
     }
 
+    public void testOrderByYear() {
+        PhysicalPlan p = optimizeAndPlan("SELECT YEAR(date) FROM test ORDER BY 1");
+        assertEquals(EsQueryExec.class, p.getClass());
+        EsQueryExec eqe = (EsQueryExec) p;
+        assertEquals(1, eqe.output().size());
+        assertEquals("YEAR(date)", eqe.output().get(0).qualifiedName());
+        assertEquals(INTEGER, eqe.output().get(0).dataType());
+        assertThat(eqe.queryContainer().toString().replaceAll("\\s+", ""),
+                endsWith("\"sort\":[{\"_script\":{\"script\":{\"source\":\"InternalSqlScriptUtils.nullSafeSortNumeric(" +
+                        "InternalSqlScriptUtils.dateTimeChrono(InternalSqlScriptUtils.docValue(doc,params.v0)," +
+                        "params.v1,params.v2))\",\"lang\":\"painless\",\"params\":{\"v0\":\"date\",\"v1\":\"Z\"," +
+                        "\"v2\":\"YEAR\"}},\"type\":\"number\",\"order\":\"asc\"}}]}"));
+    }
+
     public void testGroupByHistogramWithDate() {
         LogicalPlan p = plan("SELECT MAX(int) FROM test GROUP BY HISTOGRAM(CAST(date AS DATE), INTERVAL 2 MONTHS)");
         assertTrue(p instanceof Aggregate);