diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec index 759dc09029396..2ee23382515da 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec @@ -439,3 +439,23 @@ FROM date_nanos | WHERE millis > "2020-01-01" | STATS v = MV_SORT(VALUES(nanos), v:date_nanos [2023-10-23T13:55:01.543123456Z, 2023-10-23T13:53:55.832987654Z, 2023-10-23T13:52:55.015787878Z, 2023-10-23T13:51:54.732102837Z, 2023-10-23T13:33:34.937193000Z, 2023-10-23T12:27:28.948000000Z, 2023-10-23T12:15:03.360103847Z] ; + +Date trunc on date nanos +required_capability: date_trunc_date_nanos + +FROM date_nanos +| WHERE millis > "2020-01-01" +| EVAL yr = DATE_TRUNC(1 year, nanos), mo = DATE_TRUNC(1 month, nanos), mn = DATE_TRUNC(10 minutes, nanos), ms = DATE_TRUNC(1 millisecond, nanos) +| SORT nanos DESC +| KEEP yr, mo, mn, ms; + +yr:date_nanos | mo:date_nanos | mn:date_nanos | ms:date_nanos +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T13:50:00.000000000Z | 2023-10-23T13:55:01.543000000Z +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T13:50:00.000000000Z | 2023-10-23T13:53:55.832000000Z +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T13:50:00.000000000Z | 2023-10-23T13:52:55.015000000Z +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T13:50:00.000000000Z | 2023-10-23T13:51:54.732000000Z +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T13:30:00.000000000Z | 2023-10-23T13:33:34.937000000Z +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T12:20:00.000000000Z | 2023-10-23T12:27:28.948000000Z +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T12:10:00.000000000Z | 2023-10-23T12:15:03.360000000Z +2023-01-01T00:00:00.000000000Z | 2023-10-01T00:00:00.000000000Z | 2023-10-23T12:10:00.000000000Z | 2023-10-23T12:15:03.360000000Z +; diff --git a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncDateNanosEvaluator.java b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncDateNanosEvaluator.java new file mode 100644 index 0000000000000..2d34fc613bc74 --- /dev/null +++ b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncDateNanosEvaluator.java @@ -0,0 +1,130 @@ +// 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.expression.function.scalar.date; + +import java.lang.IllegalArgumentException; +import java.lang.Override; +import java.lang.String; +import org.elasticsearch.common.Rounding; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.LongBlock; +import org.elasticsearch.compute.data.LongVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.compute.operator.Warnings; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.xpack.esql.core.tree.Source; + +/** + * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateTrunc}. + * This class is generated. Do not edit it. + */ +public final class DateTruncDateNanosEvaluator implements EvalOperator.ExpressionEvaluator { + private final Source source; + + private final EvalOperator.ExpressionEvaluator fieldVal; + + private final Rounding.Prepared rounding; + + private final DriverContext driverContext; + + private Warnings warnings; + + public DateTruncDateNanosEvaluator(Source source, EvalOperator.ExpressionEvaluator fieldVal, + Rounding.Prepared rounding, DriverContext driverContext) { + this.source = source; + this.fieldVal = fieldVal; + this.rounding = rounding; + this.driverContext = driverContext; + } + + @Override + public Block eval(Page page) { + try (LongBlock fieldValBlock = (LongBlock) fieldVal.eval(page)) { + LongVector fieldValVector = fieldValBlock.asVector(); + if (fieldValVector == null) { + return eval(page.getPositionCount(), fieldValBlock); + } + return eval(page.getPositionCount(), fieldValVector).asBlock(); + } + } + + public LongBlock eval(int positionCount, LongBlock fieldValBlock) { + try(LongBlock.Builder result = driverContext.blockFactory().newLongBlockBuilder(positionCount)) { + position: for (int p = 0; p < positionCount; p++) { + if (fieldValBlock.isNull(p)) { + result.appendNull(); + continue position; + } + if (fieldValBlock.getValueCount(p) != 1) { + if (fieldValBlock.getValueCount(p) > 1) { + warnings().registerException(new IllegalArgumentException("single-value function encountered multi-value")); + } + result.appendNull(); + continue position; + } + result.appendLong(DateTrunc.processDateNanos(fieldValBlock.getLong(fieldValBlock.getFirstValueIndex(p)), this.rounding)); + } + return result.build(); + } + } + + public LongVector eval(int positionCount, LongVector fieldValVector) { + try(LongVector.FixedBuilder result = driverContext.blockFactory().newLongVectorFixedBuilder(positionCount)) { + position: for (int p = 0; p < positionCount; p++) { + result.appendLong(p, DateTrunc.processDateNanos(fieldValVector.getLong(p), this.rounding)); + } + return result.build(); + } + } + + @Override + public String toString() { + return "DateTruncDateNanosEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]"; + } + + @Override + public void close() { + Releasables.closeExpectNoException(fieldVal); + } + + private Warnings warnings() { + if (warnings == null) { + this.warnings = Warnings.createWarnings( + driverContext.warningsMode(), + source.source().getLineNumber(), + source.source().getColumnNumber(), + source.text() + ); + } + return warnings; + } + + static class Factory implements EvalOperator.ExpressionEvaluator.Factory { + private final Source source; + + private final EvalOperator.ExpressionEvaluator.Factory fieldVal; + + private final Rounding.Prepared rounding; + + public Factory(Source source, EvalOperator.ExpressionEvaluator.Factory fieldVal, + Rounding.Prepared rounding) { + this.source = source; + this.fieldVal = fieldVal; + this.rounding = rounding; + } + + @Override + public DateTruncDateNanosEvaluator get(DriverContext context) { + return new DateTruncDateNanosEvaluator(source, fieldVal.get(context), rounding, context); + } + + @Override + public String toString() { + return "DateTruncDateNanosEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]"; + } + } +} diff --git a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncDatetimeEvaluator.java similarity index 82% rename from x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java rename to x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncDatetimeEvaluator.java index ca6aad07c317e..b9e49dd7e795c 100644 --- a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncEvaluator.java +++ b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncDatetimeEvaluator.java @@ -22,7 +22,7 @@ * {@link EvalOperator.ExpressionEvaluator} implementation for {@link DateTrunc}. * This class is generated. Do not edit it. */ -public final class DateTruncEvaluator implements EvalOperator.ExpressionEvaluator { +public final class DateTruncDatetimeEvaluator implements EvalOperator.ExpressionEvaluator { private final Source source; private final EvalOperator.ExpressionEvaluator fieldVal; @@ -33,7 +33,7 @@ public final class DateTruncEvaluator implements EvalOperator.ExpressionEvaluato private Warnings warnings; - public DateTruncEvaluator(Source source, EvalOperator.ExpressionEvaluator fieldVal, + public DateTruncDatetimeEvaluator(Source source, EvalOperator.ExpressionEvaluator fieldVal, Rounding.Prepared rounding, DriverContext driverContext) { this.source = source; this.fieldVal = fieldVal; @@ -66,7 +66,7 @@ public LongBlock eval(int positionCount, LongBlock fieldValBlock) { result.appendNull(); continue position; } - result.appendLong(DateTrunc.process(fieldValBlock.getLong(fieldValBlock.getFirstValueIndex(p)), this.rounding)); + result.appendLong(DateTrunc.processDatetime(fieldValBlock.getLong(fieldValBlock.getFirstValueIndex(p)), this.rounding)); } return result.build(); } @@ -75,7 +75,7 @@ public LongBlock eval(int positionCount, LongBlock fieldValBlock) { public LongVector eval(int positionCount, LongVector fieldValVector) { try(LongVector.FixedBuilder result = driverContext.blockFactory().newLongVectorFixedBuilder(positionCount)) { position: for (int p = 0; p < positionCount; p++) { - result.appendLong(p, DateTrunc.process(fieldValVector.getLong(p), this.rounding)); + result.appendLong(p, DateTrunc.processDatetime(fieldValVector.getLong(p), this.rounding)); } return result.build(); } @@ -83,7 +83,7 @@ public LongVector eval(int positionCount, LongVector fieldValVector) { @Override public String toString() { - return "DateTruncEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]"; + return "DateTruncDatetimeEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]"; } @Override @@ -118,13 +118,13 @@ public Factory(Source source, EvalOperator.ExpressionEvaluator.Factory fieldVal, } @Override - public DateTruncEvaluator get(DriverContext context) { - return new DateTruncEvaluator(source, fieldVal.get(context), rounding, context); + public DateTruncDatetimeEvaluator get(DriverContext context) { + return new DateTruncDatetimeEvaluator(source, fieldVal.get(context), rounding, context); } @Override public String toString() { - return "DateTruncEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]"; + return "DateTruncDatetimeEvaluator[" + "fieldVal=" + fieldVal + ", rounding=" + rounding + "]"; } } } 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 cb0237011e03d..d7e0f4c13e1b4 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 @@ -329,6 +329,11 @@ public enum Cap { */ LEAST_GREATEST_FOR_DATENANOS(EsqlCorePlugin.DATE_NANOS_FEATURE_FLAG), + /** + * Support for date_trunc function on date nanos type + */ + DATE_TRUNC_DATE_NANOS(EsqlCorePlugin.DATE_NANOS_FEATURE_FLAG), + /** * support aggregations on date nanos */ diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Bucket.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Bucket.java index 3357b2abf0e0f..9e40b85fd6590 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Bucket.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Bucket.java @@ -252,7 +252,7 @@ public ExpressionEvaluator.Factory toEvaluator(ToEvaluator toEvaluator) { assert DataType.isTemporalAmount(buckets.dataType()) : "Unexpected span data type [" + buckets.dataType() + "]"; preparedRounding = DateTrunc.createRounding(buckets.fold(), DEFAULT_TZ); } - return DateTrunc.evaluator(source(), toEvaluator.apply(field), preparedRounding); + return DateTrunc.evaluator(field.dataType(), source(), toEvaluator.apply(field), preparedRounding); } if (field.dataType().isNumeric()) { double roundTo; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTrunc.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTrunc.java index 35a705f418906..6e38d72500840 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTrunc.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTrunc.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.compute.ann.Evaluator; import org.elasticsearch.compute.ann.Fixed; import org.elasticsearch.compute.operator.EvalOperator.ExpressionEvaluator; @@ -31,12 +32,14 @@ import java.time.ZoneId; import java.time.ZoneOffset; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; -import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isDate; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; public class DateTrunc extends EsqlScalarFunction { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( @@ -45,6 +48,15 @@ public class DateTrunc extends EsqlScalarFunction { DateTrunc::new ); + @FunctionalInterface + public interface DateTruncFactoryProvider { + ExpressionEvaluator.Factory apply(Source source, ExpressionEvaluator.Factory lhs, Rounding.Prepared rounding); + } + + private static final Map evaluatorMap = Map.ofEntries( + Map.entry(DATETIME, DateTruncDatetimeEvaluator.Factory::new), + Map.entry(DATE_NANOS, DateTruncDateNanosEvaluator.Factory::new) + ); private final Expression interval; private final Expression timestampField; protected static final ZoneId DEFAULT_TZ = ZoneOffset.UTC; @@ -108,20 +120,28 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } + String operationName = sourceText(); return isType(interval, DataType::isTemporalAmount, sourceText(), FIRST, "dateperiod", "timeduration").and( - isDate(timestampField, sourceText(), SECOND) + isType(timestampField, evaluatorMap::containsKey, operationName, SECOND, "date_nanos or datetime") ); } public DataType dataType() { - return DataType.DATETIME; + // Default to DATETIME in the case of nulls. This mimics the behavior before DATE_NANOS support + return timestampField.dataType() == DataType.NULL ? DATETIME : timestampField.dataType(); } - @Evaluator - static long process(long fieldVal, @Fixed Rounding.Prepared rounding) { + @Evaluator(extraName = "Datetime") + static long processDatetime(long fieldVal, @Fixed Rounding.Prepared rounding) { return rounding.round(fieldVal); } + @Evaluator(extraName = "DateNanos") + static long processDateNanos(long fieldVal, @Fixed Rounding.Prepared rounding) { + // Currently, ES|QL doesn't support rounding to sub-millisecond values, so it's safe to cast before rounding. + return DateUtils.toNanoSeconds(rounding.round(DateUtils.toMilliSeconds(fieldVal))); + } + @Override public Expression replaceChildren(List newChildren) { return new DateTrunc(source(), newChildren.get(0), newChildren.get(1)); @@ -214,14 +234,15 @@ public ExpressionEvaluator.Factory toEvaluator(ToEvaluator toEvaluator) { "Function [" + sourceText() + "] has invalid interval [" + interval.sourceText() + "]. " + e.getMessage() ); } - return evaluator(source(), fieldEvaluator, DateTrunc.createRounding(foldedInterval, DEFAULT_TZ)); + return evaluator(dataType(), source(), fieldEvaluator, DateTrunc.createRounding(foldedInterval, DEFAULT_TZ)); } public static ExpressionEvaluator.Factory evaluator( + DataType forType, Source source, ExpressionEvaluator.Factory fieldEvaluator, Rounding.Prepared rounding ) { - return new DateTruncEvaluator.Factory(source, fieldEvaluator, rounding); + return evaluatorMap.get(forType).apply(source, fieldEvaluator, rounding); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java index f7991a48a5744..c1b2adddfc838 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java @@ -1118,21 +1118,24 @@ public void testDateTruncOnInt() { verifyUnsupported(""" from test | eval date_trunc(1 month, int) - """, "second argument of [date_trunc(1 month, int)] must be [datetime], found value [int] type [integer]"); + """, "second argument of [date_trunc(1 month, int)] must be [date_nanos or datetime], found value [int] type [integer]"); } public void testDateTruncOnFloat() { verifyUnsupported(""" from test | eval date_trunc(1 month, float) - """, "second argument of [date_trunc(1 month, float)] must be [datetime], found value [float] type [double]"); + """, "second argument of [date_trunc(1 month, float)] must be [date_nanos or datetime], found value [float] type [double]"); } public void testDateTruncOnText() { - verifyUnsupported(""" - from test - | eval date_trunc(1 month, keyword) - """, "second argument of [date_trunc(1 month, keyword)] must be [datetime], found value [keyword] type [keyword]"); + verifyUnsupported( + """ + from test + | eval date_trunc(1 month, keyword) + """, + "second argument of [date_trunc(1 month, keyword)] must be [date_nanos or datetime], found value [keyword] type [keyword]" + ); } public void testDateTruncWithNumericInterval() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/grouping/BucketTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/grouping/BucketTests.java index a26504b8ced9a..7e7d91cdf76f4 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/grouping/BucketTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/grouping/BucketTests.java @@ -87,7 +87,8 @@ private static void dateCases(List suppliers, String name, Lon args.add(dateBound("to", toType, "2023-03-01T09:00:00.00Z")); return new TestCaseSupplier.TestCase( args, - "DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding[DAY_OF_MONTH in Z][fixed to midnight]]", + "DateTruncDatetimeEvaluator[fieldVal=Attribute[channel=0], " + + "rounding=Rounding[DAY_OF_MONTH in Z][fixed to midnight]]", DataType.DATETIME, resultsMatcher(args) ); @@ -101,7 +102,7 @@ private static void dateCases(List suppliers, String name, Lon args.add(dateBound("to", toType, "2023-02-17T12:00:00Z")); return new TestCaseSupplier.TestCase( args, - "DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding[3600000 in Z][fixed]]", + "DateTruncDatetimeEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding[3600000 in Z][fixed]]", DataType.DATETIME, equalTo(Rounding.builder(Rounding.DateTimeUnit.HOUR_OF_DAY).build().prepareForUnknown().round(date.getAsLong())) ); @@ -134,7 +135,7 @@ private static void dateCasesWithSpan( args.add(new TestCaseSupplier.TypedData(span, spanType, "buckets").forceLiteral()); return new TestCaseSupplier.TestCase( args, - "DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding" + spanStr + "]", + "DateTruncDatetimeEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding" + spanStr + "]", DataType.DATETIME, resultsMatcher(args) ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java index 5af5c8e493177..b5e89fc41f368 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java @@ -15,7 +15,7 @@ import java.time.Period; import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.createRounding; -import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.process; +import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.processDatetime; import static org.hamcrest.Matchers.containsString; /** @@ -97,10 +97,13 @@ public void testCreateRoundingNullInterval() { public void testDateTruncFunction() { long ts = toMillis("2023-02-17T10:25:33.38Z"); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Period.ofDays(-1)))); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> processDatetime(ts, createRounding(Period.ofDays(-1))) + ); assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); - e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Duration.ofHours(-1)))); + e = expectThrows(IllegalArgumentException.class, () -> processDatetime(ts, createRounding(Duration.ofHours(-1)))); assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java index 0e4968cc2a504..2403900064645 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java @@ -10,6 +10,7 @@ import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; @@ -20,6 +21,7 @@ import java.time.Duration; import java.time.Instant; import java.time.Period; +import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; @@ -37,56 +39,84 @@ public DateTruncTests(@Name("TestCase") Supplier test @ParametersFactory public static Iterable parameters() { long ts = toMillis("2023-02-17T10:25:33.38Z"); - List suppliers = List.of( - ofDatePeriod(Period.ofDays(1), ts, "2023-02-17T00:00:00.00Z"), - ofDatePeriod(Period.ofMonths(1), ts, "2023-02-01T00:00:00.00Z"), - ofDatePeriod(Period.ofYears(1), ts, "2023-01-01T00:00:00.00Z"), - ofDatePeriod(Period.ofDays(10), ts, "2023-02-12T00:00:00.00Z"), - // 7 days period should return weekly rounding - ofDatePeriod(Period.ofDays(7), ts, "2023-02-13T00:00:00.00Z"), - // 3 months period should return quarterly - ofDatePeriod(Period.ofMonths(3), ts, "2023-01-01T00:00:00.00Z"), - ofDuration(Duration.ofHours(1), ts, "2023-02-17T10:00:00.00Z"), - ofDuration(Duration.ofMinutes(1), ts, "2023-02-17T10:25:00.00Z"), - ofDuration(Duration.ofSeconds(1), ts, "2023-02-17T10:25:33.00Z"), - ofDuration(Duration.ofHours(3), ts, "2023-02-17T09:00:00.00Z"), - ofDuration(Duration.ofMinutes(15), ts, "2023-02-17T10:15:00.00Z"), - ofDuration(Duration.ofSeconds(30), ts, "2023-02-17T10:25:30.00Z"), - randomSecond() - ); + List suppliers = new ArrayList<>(); + suppliers.addAll(ofDatePeriod(Period.ofDays(1), ts, "2023-02-17T00:00:00.00Z")); + suppliers.addAll(ofDatePeriod(Period.ofMonths(1), ts, "2023-02-01T00:00:00.00Z")); + suppliers.addAll(ofDatePeriod(Period.ofYears(1), ts, "2023-01-01T00:00:00.00Z")); + suppliers.addAll(ofDatePeriod(Period.ofDays(10), ts, "2023-02-12T00:00:00.00Z")); + // 7 days period should return weekly rounding + suppliers.addAll(ofDatePeriod(Period.ofDays(7), ts, "2023-02-13T00:00:00.00Z")); + // 3 months period should return quarterly + suppliers.addAll(ofDatePeriod(Period.ofMonths(3), ts, "2023-01-01T00:00:00.00Z")); + suppliers.addAll(ofDuration(Duration.ofHours(1), ts, "2023-02-17T10:00:00.00Z")); + suppliers.addAll(ofDuration(Duration.ofMinutes(1), ts, "2023-02-17T10:25:00.00Z")); + suppliers.addAll(ofDuration(Duration.ofSeconds(1), ts, "2023-02-17T10:25:33.00Z")); + suppliers.addAll(ofDuration(Duration.ofHours(3), ts, "2023-02-17T09:00:00.00Z")); + suppliers.addAll(ofDuration(Duration.ofMinutes(15), ts, "2023-02-17T10:15:00.00Z")); + suppliers.addAll(ofDuration(Duration.ofSeconds(30), ts, "2023-02-17T10:25:30.00Z")); + suppliers.add(randomSecond()); + return parameterSuppliersFromTypedDataWithDefaultChecks(true, suppliers, (v, p) -> switch (p) { case 0 -> "dateperiod or timeduration"; - case 1 -> "datetime"; + case 1 -> "date_nanos or datetime"; default -> null; }); } - private static TestCaseSupplier ofDatePeriod(Period period, long value, String expectedDate) { - return new TestCaseSupplier( - List.of(DataType.DATE_PERIOD, DataType.DATETIME), - () -> new TestCaseSupplier.TestCase( - List.of( - new TestCaseSupplier.TypedData(period, DataType.DATE_PERIOD, "interval").forceLiteral(), - new TestCaseSupplier.TypedData(value, DataType.DATETIME, "date") - ), - Matchers.startsWith("DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), - DataType.DATETIME, - equalTo(toMillis(expectedDate)) + private static List ofDatePeriod(Period period, long value, String expectedDate) { + return List.of( + new TestCaseSupplier( + List.of(DataType.DATE_PERIOD, DataType.DATETIME), + () -> new TestCaseSupplier.TestCase( + List.of( + new TestCaseSupplier.TypedData(period, DataType.DATE_PERIOD, "interval").forceLiteral(), + new TestCaseSupplier.TypedData(value, DataType.DATETIME, "date") + ), + Matchers.startsWith("DateTruncDatetimeEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), + DataType.DATETIME, + equalTo(toMillis(expectedDate)) + ) + ), + new TestCaseSupplier( + List.of(DataType.DATE_PERIOD, DataType.DATE_NANOS), + () -> new TestCaseSupplier.TestCase( + List.of( + new TestCaseSupplier.TypedData(period, DataType.DATE_PERIOD, "interval").forceLiteral(), + new TestCaseSupplier.TypedData(DateUtils.toNanoSeconds(value), DataType.DATE_NANOS, "date") + ), + Matchers.startsWith("DateTruncDateNanosEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), + DataType.DATE_NANOS, + equalTo(toNanos(expectedDate)) + ) ) ); } - private static TestCaseSupplier ofDuration(Duration duration, long value, String expectedDate) { - return new TestCaseSupplier( - List.of(DataType.TIME_DURATION, DataType.DATETIME), - () -> new TestCaseSupplier.TestCase( - List.of( - new TestCaseSupplier.TypedData(duration, DataType.TIME_DURATION, "interval").forceLiteral(), - new TestCaseSupplier.TypedData(value, DataType.DATETIME, "date") - ), - Matchers.startsWith("DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), - DataType.DATETIME, - equalTo(toMillis(expectedDate)) + private static List ofDuration(Duration duration, long value, String expectedDate) { + return List.of( + new TestCaseSupplier( + List.of(DataType.TIME_DURATION, DataType.DATETIME), + () -> new TestCaseSupplier.TestCase( + List.of( + new TestCaseSupplier.TypedData(duration, DataType.TIME_DURATION, "interval").forceLiteral(), + new TestCaseSupplier.TypedData(value, DataType.DATETIME, "date") + ), + Matchers.startsWith("DateTruncDatetimeEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), + DataType.DATETIME, + equalTo(toMillis(expectedDate)) + ) + ), + new TestCaseSupplier( + List.of(DataType.TIME_DURATION, DataType.DATE_NANOS), + () -> new TestCaseSupplier.TestCase( + List.of( + new TestCaseSupplier.TypedData(duration, DataType.TIME_DURATION, "interval").forceLiteral(), + new TestCaseSupplier.TypedData(DateUtils.toNanoSeconds(value), DataType.DATE_NANOS, "date") + ), + Matchers.startsWith("DateTruncDateNanosEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), + DataType.DATE_NANOS, + equalTo(toNanos(expectedDate)) + ) ) ); } @@ -109,7 +139,7 @@ private static TestCaseSupplier randomSecond() { new TestCaseSupplier.TypedData(Duration.ofSeconds(1), DataType.TIME_DURATION, "interval"), new TestCaseSupplier.TypedData(toMillis(dateFragment + ".38Z"), DataType.DATETIME, "date") ), - "DateTruncEvaluator[date=Attribute[channel=1], interval=Attribute[channel=0]]", + "DateTruncDatetimeEvaluator[date=Attribute[channel=1], interval=Attribute[channel=0]]", DataType.DATETIME, equalTo(toMillis(dateFragment + ".00Z")) ); @@ -124,6 +154,10 @@ private static long toMillis(String timestamp) { return Instant.parse(timestamp).toEpochMilli(); } + private static long toNanos(String timestamp) { + return DateUtils.toLong(Instant.parse(timestamp)); + } + @Override protected Expression build(Source source, List args) { return new DateTrunc(source, args.get(0), args.get(1));