From 450788e0d4dca21d5189140bacdbe626b0932572 Mon Sep 17 00:00:00 2001 From: Shmuel Hanoch Date: Thu, 21 Aug 2025 13:18:33 +0300 Subject: [PATCH 1/8] Basic support for DATE_RANGE field type in ESQL --- .../index/mapper/BlockLoader.java | 8 + .../index/mapper/RangeFieldMapper.java | 95 ++++++++ .../esql_date_range_created_version.csv | 1 + .../elasticsearch/index/mapper/TestBlock.java | 70 ++++++ .../xpack/esql/core/type/DataType.java | 3 + .../compute/data/BlockFactory.java | 16 ++ .../compute/data/BlockUtils.java | 8 + .../compute/data/ConstantNullBlock.java | 13 +- .../compute/data/DateRangeArrayBlock.java | 218 ++++++++++++++++++ .../compute/data/DateRangeBlock.java | 73 ++++++ .../compute/data/DateRangeBlockBuilder.java | 179 ++++++++++++++ .../compute/data/ElementType.java | 6 +- .../read/DelegatingBlockLoaderFactory.java | 5 + .../compute/operator/lookup/QueryList.java | 1 + .../data/BlockBuilderCopyFromTests.java | 1 + .../compute/data/BlockBuilderTests.java | 5 +- .../compute/data/CompositeBlockTests.java | 6 +- .../compute/data/VectorBuilderTests.java | 2 +- .../compute/data/VectorFixedBuilderTests.java | 2 +- .../mvdedupe/MultivalueDedupeTests.java | 3 +- .../compute/test/BlockTestUtils.java | 14 ++ .../compute/test/RandomBlock.java | 14 ++ .../xpack/esql/qa/single_node/RestEsqlIT.java | 2 + .../qa/rest/AllSupportedFieldsTestCase.java | 15 +- .../elasticsearch/xpack/esql/CsvAssert.java | 7 + .../xpack/esql/CsvTestUtils.java | 7 +- .../xpack/esql/CsvTestsDataLoader.java | 4 +- .../xpack/esql/EsqlTestUtils.java | 7 + .../src/main/resources/data/ages.csv | 20 +- .../src/main/resources/data/date_ranges.csv | 14 ++ .../src/main/resources/data/decades.csv | 28 +-- .../src/main/resources/date.csv-spec | 21 ++ .../main/resources/mapping-date_ranges.json | 13 ++ .../xpack/esql/action/PositionToXContent.java | 11 + .../xpack/esql/action/ResponseValueUtils.java | 7 + .../xpack/esql/analysis/PreAnalyzer.java | 9 +- .../esql/enrich/EnrichPolicyResolver.java | 12 +- .../expression/function/aggregate/Absent.java | 4 +- .../expression/function/aggregate/Count.java | 4 +- .../function/aggregate/Present.java | 4 +- .../function/scalar/convert/ToDateRange.java | 74 ++++++ .../function/scalar/convert/ToString.java | 4 +- .../ToStringFromDateRangeEvaluator.java | 95 ++++++++ .../function/scalar/nulls/Coalesce.java | 2 +- .../xpack/esql/planner/AggregateMapper.java | 2 +- .../esql/planner/LocalExecutionPlanner.java | 2 +- .../xpack/esql/planner/PlannerUtils.java | 1 + .../xpack/esql/session/EsqlSession.java | 2 + .../xpack/esql/session/IndexResolver.java | 15 +- .../esql/type/EsqlDataTypeConverter.java | 25 ++ .../xpack/esql/SerializationTestUtils.java | 8 + .../esql/action/EsqlQueryResponseTests.java | 7 + .../xpack/esql/analysis/AnalyzerTests.java | 21 +- .../expression/function/TestCaseSupplier.java | 31 +++ .../scalar/convert/ToStringTests.java | 9 + ...bstractPhysicalPlanSerializationTests.java | 2 + .../esql/type/EsqlDataTypeRegistryTests.java | 2 +- 57 files changed, 1175 insertions(+), 59 deletions(-) create mode 100644 server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeArrayBlock.java create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlockBuilder.java create mode 100644 x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/date_ranges.csv create mode 100644 x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-date_ranges.json create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDateRange.java create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BlockLoader.java b/server/src/main/java/org/elasticsearch/index/mapper/BlockLoader.java index 5f2bd15abaa34..215b10e22c71e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/BlockLoader.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/BlockLoader.java @@ -496,6 +496,8 @@ interface BlockFactory { SortedSetOrdinalsBuilder sortedSetOrdinalsBuilder(SortedSetDocValues ordinals, int count); AggregateMetricDoubleBuilder aggregateMetricDoubleBuilder(int count); + + DateRangeBuilder dateRangeBuilder(int count); } /** @@ -623,4 +625,10 @@ interface AggregateMetricDoubleBuilder extends Builder { IntBuilder count(); } + + interface DateRangeBuilder extends Builder { + LongBuilder from(); + + LongBuilder to(); + } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java index ee891eb6a6c1a..87c53e33523a6 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/RangeFieldMapper.java @@ -9,10 +9,13 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Query; import org.apache.lucene.search.SortField; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.TransportVersion; import org.elasticsearch.common.Explicit; import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.network.InetAddresses; @@ -61,6 +64,8 @@ public class RangeFieldMapper extends FieldMapper { public static final boolean DEFAULT_INCLUDE_UPPER = true; public static final boolean DEFAULT_INCLUDE_LOWER = true; + public static final TransportVersion ESQL_DATE_RANGE_CREATED_VERSION = TransportVersion.fromName("esql_date_range_created_version"); + public static class Defaults { public static final DateFormatter DATE_FORMATTER = DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; public static final Locale LOCALE = DateFieldMapper.DEFAULT_LOCALE; @@ -342,6 +347,96 @@ public Query rangeQuery( context ); } + + public static class DateRangeDocValuesLoader extends BlockDocValuesReader.DocValuesBlockLoader { + private final String fieldName; + + public DateRangeDocValuesLoader(String fieldName) { + this.fieldName = fieldName; + } + + @Override + public Builder builder(BlockFactory factory, int expectedCount) { + return factory.longsFromDocValues(expectedCount); + } + + @Override + public AllReader reader(LeafReaderContext context) throws IOException { + var docValues = context.reader().getBinaryDocValues(fieldName); + return new DateRangeDocValuesReader(docValues); + } + } + + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (hasDocValues()) { + return new DateRangeDocValuesLoader(name()); + } + throw new IllegalStateException("Cannot load blocks without doc values"); + } + } + + public static class DateRangeDocValuesReader extends BlockDocValuesReader { + private final BytesRef spare = new BytesRef(); + + private final BinaryDocValues numericDocValues; + + public DateRangeDocValuesReader(BinaryDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + private int docId = -1; + + @Override + protected int docId() { + return docId; + } + + @Override + public String toString() { + return "BlockDocValuesReader.DateRangeDocValuesReader"; + } + + @Override + public BlockLoader.Block read(BlockLoader.BlockFactory factory, BlockLoader.Docs docs, int offset, boolean nullsFiltered) + throws IOException { + try (BlockLoader.DateRangeBuilder builder = factory.dateRangeBuilder(docs.count() - offset)) { + int lastDoc = -1; + for (int i = offset; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + } else { + BytesRef ref = numericDocValues.binaryValue(); + var ranges = BinaryRangeUtil.decodeLongRanges(ref); + for (var range : ranges) { + lastDoc = doc; + this.docId = doc; + builder.from().appendLong((long) range.from); + builder.to().appendLong((long) range.to); + } + } + } + return builder.build(); + } + } + + @Override + public void read(int doc, BlockLoader.StoredFields storedFields, BlockLoader.Builder builder) throws IOException { + var blockBuilder = (BlockLoader.DateRangeBuilder) builder; + this.docId = doc; + if (false == numericDocValues.advanceExact(doc)) { + blockBuilder.appendNull(); + } else { + var range = BinaryRangeUtil.decodeLongRanges(numericDocValues.binaryValue()); + assert range.size() == 1 : "stored fields should only have a single range"; + blockBuilder.from().appendLong((long) range.getFirst().from); + blockBuilder.to().appendLong((long) range.getFirst().to); + } + } } private final RangeType type; diff --git a/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv b/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv new file mode 100644 index 0000000000000..ce3efd6a04801 --- /dev/null +++ b/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv @@ -0,0 +1 @@ +9190000 diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/TestBlock.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/TestBlock.java index 7f4dc19d8f433..465c618fae610 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/TestBlock.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/TestBlock.java @@ -414,9 +414,15 @@ public SortedSetOrdinalBuilder appendOrd(int value) { return new SortedSetOrdinalBuilder(); } + @Override public BlockLoader.AggregateMetricDoubleBuilder aggregateMetricDoubleBuilder(int expectedSize) { return new AggregateMetricDoubleBlockBuilder(expectedSize); } + + @Override + public BlockLoader.DateRangeBuilder dateRangeBuilder(int expectedSize) { + return new DateRangeBuilder(expectedSize); + } }; } @@ -624,4 +630,68 @@ public void close() { } } + + public static class DateRangeBuilder implements BlockLoader.DateRangeBuilder { + private final LongBuilder from; + private final LongBuilder to; + + DateRangeBuilder(int expectedSize) { + from = new LongBuilder(expectedSize); + to = new LongBuilder(expectedSize); + } + + @Override + public BlockLoader.LongBuilder from() { + return from; + } + + @Override + public BlockLoader.LongBuilder to() { + return to; + } + + @Override + public BlockLoader.Block build() { + var fromBlock = from.build(); + var toBlock = to.build(); + assert fromBlock.size() == toBlock.size(); + var values = new ArrayList<>(fromBlock.size()); + for (int i = 0; i < fromBlock.size(); i++) { + values.add(List.of(fromBlock.values.get(i), toBlock.values.get(i))); + } + return new TestBlock(values); + } + + @Override + public BlockLoader.Builder appendNull() { + throw new UnsupportedOperationException(); + } + + @Override + public BlockLoader.Builder beginPositionEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public BlockLoader.Builder endPositionEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + + } + + private static class LongBuilder extends TestBlock.Builder implements BlockLoader.LongBuilder { + private LongBuilder(int expectedSize) { + super(expectedSize); + } + + @Override + public BlockLoader.LongBuilder appendLong(long value) { + add(value); + return this; + } + } + }; } diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java index b7a3204c43692..7335d3965e87c 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java @@ -33,6 +33,7 @@ import java.util.stream.Collectors; import static java.util.stream.Collectors.toMap; +import static org.elasticsearch.index.mapper.RangeFieldMapper.ESQL_DATE_RANGE_CREATED_VERSION; /** * This enum represents data types the ES|QL query processing layer is able to @@ -270,6 +271,7 @@ public enum DataType implements Writeable { * Nanosecond precision date, stored as a 64-bit signed number. */ DATE_NANOS(builder().esType("date_nanos").estimatedSize(Long.BYTES).docValues().supportedOnAllNodes()), + DATE_RANGE(builder().esType("date_range").estimatedSize(2 * Long.BYTES).docValues().supportedOn(ESQL_DATE_RANGE_CREATED_VERSION)), /** * IP addresses. IPv4 address are always * embedded @@ -982,5 +984,6 @@ private static class DataTypesTransportVersions { public static final TransportVersion ESQL_AGGREGATE_METRIC_DOUBLE_CREATED_VERSION = TransportVersion.fromName( "esql_aggregate_metric_double_created_version" ); + } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockFactory.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockFactory.java index 53cb0a72f86fc..16217976e30d2 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockFactory.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockFactory.java @@ -481,6 +481,22 @@ public final AggregateMetricDoubleBlock newAggregateMetricDoubleBlock( return new AggregateMetricDoubleArrayBlock(min, max, sum, count); } + public DateRangeBlockBuilder newDateRangeBlockBuilder(int estimatedSize) { + return new DateRangeBlockBuilder(estimatedSize, this); + } + + public DateRangeBlock newConstantDateRangeBlock(DateRangeBlockBuilder.DateRangeLiteral value, int positions) { + var fromBuilder = newConstantLongBlockWith(value.from(), positions); + var toBuilder = newConstantLongBlockWith(value.to(), positions); + return new DateRangeArrayBlock(fromBuilder, toBuilder); + } + + public DateRangeBlock newDateRangeBlock(long[] fromValues, long[] toValues, int positions) { + var from = newLongArrayVector(fromValues, positions).asBlock(); + var to = newLongArrayVector(toValues, positions).asBlock(); + return new DateRangeArrayBlock(from, to); + } + /** * Returns the maximum number of bytes that a Block should be backed by a primitive array before switching to using BigArrays. */ diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockUtils.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockUtils.java index fa5f139c5c6a1..65d76cb0b2df9 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockUtils.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/BlockUtils.java @@ -224,6 +224,7 @@ public static void appendValue(Block.Builder builder, Object val, ElementType ty case DOUBLE -> ((DoubleBlock.Builder) builder).appendDouble((Double) val); case BOOLEAN -> ((BooleanBlock.Builder) builder).appendBoolean((Boolean) val); case AGGREGATE_METRIC_DOUBLE -> ((AggregateMetricDoubleBlockBuilder) builder).appendLiteral((AggregateMetricDoubleLiteral) val); + case DATE_RANGE -> ((DateRangeBlockBuilder) builder).appendDateRange((DateRangeBlockBuilder.DateRangeLiteral) val); default -> throw new UnsupportedOperationException("unsupported element type [" + type + "]"); } } @@ -253,6 +254,7 @@ private static Block constantBlock(BlockFactory blockFactory, ElementType type, case BOOLEAN -> blockFactory.newConstantBooleanBlockWith((boolean) val, size); case AGGREGATE_METRIC_DOUBLE -> blockFactory.newConstantAggregateMetricDoubleBlock((AggregateMetricDoubleLiteral) val, size); case FLOAT -> blockFactory.newConstantFloatBlockWith((float) val, size); + case DATE_RANGE -> blockFactory.newConstantDateRangeBlock((DateRangeBlockBuilder.DateRangeLiteral) val, size); default -> throw new UnsupportedOperationException("unsupported element type [" + type + "]"); }; } @@ -306,6 +308,12 @@ yield new AggregateMetricDoubleLiteral( aggBlock.countBlock().getInt(offset) ); } + case DATE_RANGE -> { + DateRangeBlock b = (DateRangeBlock) block; + LongBlock fromBlock = b.getFromBlock(); + LongBlock toBlock = b.getToBlock(); + yield new DateRangeBlockBuilder.DateRangeLiteral(fromBlock.getLong(offset), toBlock.getLong(offset)); + } case UNKNOWN -> throw new IllegalArgumentException("can't read values from [" + block + "]"); }; } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ConstantNullBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ConstantNullBlock.java index 22f4809100014..31cd60b18a227 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ConstantNullBlock.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ConstantNullBlock.java @@ -26,7 +26,8 @@ public final class ConstantNullBlock extends AbstractNonThreadSafeRefCounted FloatBlock, DoubleBlock, BytesRefBlock, - AggregateMetricDoubleBlock { + AggregateMetricDoubleBlock, + DateRangeBlock { private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(ConstantNullBlock.class); private final int positionCount; @@ -118,6 +119,16 @@ public ConstantNullBlock expand() { return this; } + @Override + public LongBlock getFromBlock() { + return this; + } + + @Override + public LongBlock getToBlock() { + return this; + } + @Override public long ramBytesUsed() { return BASE_RAM_BYTES_USED; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeArrayBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeArrayBlock.java new file mode 100644 index 0000000000000..7852d33240685 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeArrayBlock.java @@ -0,0 +1,218 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.ReleasableIterator; +import org.elasticsearch.core.Releasables; + +import java.io.IOException; + +public final class DateRangeArrayBlock extends AbstractNonThreadSafeRefCounted implements DateRangeBlock { + private final LongBlock fromBlock; + private final LongBlock toBlock; + + public DateRangeArrayBlock(LongBlock fromBlock, LongBlock toBlock) { + this.fromBlock = fromBlock; + this.toBlock = toBlock; + } + + @Override + public LongBlock getFromBlock() { + return fromBlock; + } + + @Override + public LongBlock getToBlock() { + return toBlock; + } + + @Override + protected void closeInternal() { + Releasables.close(fromBlock, toBlock); + } + + @Override + public Vector asVector() { + return null; + } + + @Override + public int getTotalValueCount() { + return fromBlock.getTotalValueCount() + toBlock.getTotalValueCount(); + } + + @Override + public int getPositionCount() { + return fromBlock.getPositionCount(); + } + + @Override + public int getFirstValueIndex(int position) { + return fromBlock.getFirstValueIndex(position); + } + + @Override + public int getValueCount(int position) { + return Math.max(fromBlock.getValueCount(position), toBlock.getValueCount(position)); + } + + @Override + public ElementType elementType() { + return ElementType.DATE_RANGE; + } + + @Override + public BlockFactory blockFactory() { + return fromBlock.blockFactory(); + } + + @Override + public void allowPassingToDifferentDriver() { + fromBlock.allowPassingToDifferentDriver(); + toBlock.allowPassingToDifferentDriver(); + } + + @Override + public boolean isNull(int position) { + return fromBlock.isNull(position) || toBlock.isNull(position); + } + + @Override + public boolean mayHaveNulls() { + return fromBlock.mayHaveNulls() || toBlock.mayHaveNulls(); + } + + @Override + public boolean areAllValuesNull() { + return fromBlock.areAllValuesNull() && toBlock.areAllValuesNull(); + } + + @Override + public boolean mayHaveMultivaluedFields() { + return fromBlock.mayHaveMultivaluedFields() || toBlock.mayHaveMultivaluedFields(); + } + + @Override + public boolean doesHaveMultivaluedFields() { + return fromBlock.doesHaveMultivaluedFields() || toBlock.doesHaveMultivaluedFields(); + } + + @Override + public DateRangeBlock filter(int... positions) { + DateRangeBlock result = null; + LongBlock newFromBlock = null; + LongBlock newToBlock = null; + try { + newFromBlock = fromBlock.filter(positions); + newToBlock = toBlock.filter(positions); + result = new DateRangeArrayBlock(newFromBlock, newToBlock); + return result; + } finally { + if (result == null) { + Releasables.close(newFromBlock, newToBlock); + } + } + } + + @Override + public DateRangeBlock keepMask(BooleanVector mask) { + DateRangeBlock result = null; + LongBlock newFromBlock = null; + LongBlock newToBlock = null; + try { + newFromBlock = fromBlock.keepMask(mask); + newToBlock = toBlock.keepMask(mask); + result = new DateRangeArrayBlock(newFromBlock, newToBlock); + return result; + } finally { + if (result == null) { + Releasables.close(newFromBlock, newToBlock); + } + } + } + + @Override + public ReleasableIterator lookup(IntBlock positions, ByteSizeValue targetBlockSize) { + // TODO: support + throw new UnsupportedOperationException("can't lookup values from DateRangeBlock"); + } + + @Override + public MvOrdering mvOrdering() { + // TODO: determine based on sub-blocks + return MvOrdering.UNORDERED; + } + + @Override + public DateRangeBlock expand() { + this.incRef(); + return this; + } + + @Override + public Block deepCopy(BlockFactory blockFactory) { + DateRangeBlock ret = null; + LongBlock newFromBlock = null; + LongBlock newToBlock = null; + try { + newFromBlock = fromBlock.deepCopy(blockFactory); + newToBlock = toBlock.deepCopy(blockFactory); + ret = new DateRangeArrayBlock(newFromBlock, newToBlock); + return ret; + } finally { + if (ret == null) { + Releasables.close(newFromBlock, newToBlock); + } + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + fromBlock.writeTo(out); + toBlock.writeTo(out); + } + + public static Block readFrom(StreamInput in) throws IOException { + boolean success = false; + LongBlock from = null; + LongBlock to = null; + BlockStreamInput blockStreamInput = (BlockStreamInput) in; + try { + from = LongBlock.readFrom(blockStreamInput); + to = LongBlock.readFrom(blockStreamInput); + var result = new DateRangeArrayBlock(from, to); + success = true; + return result; + } finally { + if (success == false) { + Releasables.close(from, to); + } + } + } + + @Override + public long ramBytesUsed() { + return fromBlock.ramBytesUsed() + toBlock.ramBytesUsed(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof DateRangeArrayBlock that) { + return DateRangeBlock.equals(this, that); + } + return false; + } + + @Override + public int hashCode() { + return DateRangeBlock.hash(this); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java new file mode 100644 index 0000000000000..d8f93f940b976 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java @@ -0,0 +1,73 @@ +/* + * 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.compute.data; + +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.ReleasableIterator; + +/** + * Block that stores aggregate_metric_double values. + */ +public sealed interface DateRangeBlock extends Block permits DateRangeArrayBlock, ConstantNullBlock { + @Override + DateRangeBlock filter(int... positions); + + @Override + DateRangeBlock keepMask(BooleanVector mask); + + @Override + ReleasableIterator lookup(IntBlock positions, ByteSizeValue targetBlockSize); + + @Override + DateRangeBlock expand(); + + /** + * Returns {@code true} if the given blocks are equal to each other, otherwise {@code false}. + * Two blocks are considered equal if they have the same position count, and contain the same + * values (including absent null values) in the same order. This definition ensures that the + * equals method works properly across different implementations of the AggregateMetricDoubleBlock interface. + */ + static boolean equals(DateRangeBlock lhs, DateRangeBlock rhs) { + if (lhs == rhs) return true; + if (lhs.getPositionCount() != rhs.getPositionCount()) return false; + return LongBlock.equals(lhs.getFromBlock(), rhs.getFromBlock()) && LongBlock.equals(lhs.getToBlock(), rhs.getToBlock()); + } + + static int hash(DateRangeBlock block) { + final int positions = block.getPositionCount(); + int ret = 1; + for (int pos = 0; pos < positions; pos++) { + if (block.isNull(pos)) { + ret = 31 * ret - 1; + } else { + final int valueCount = block.getValueCount(pos); + ret = 31 * ret + valueCount; + final int firstValueIdx = block.getFirstValueIndex(pos); + for (int valueIndex = 0; valueIndex < valueCount; valueIndex++) { + ret *= 31; + ret += extractHashFrom(block.getFromBlock(), firstValueIdx, valueIndex); + ret *= 31; + ret += extractHashFrom(block.getToBlock(), firstValueIdx, valueIndex); + } + } + } + return ret; + } + + private static int extractHashFrom(LongBlock b, int firstValueIdx, int valueIndex) { + return b.isNull(firstValueIdx + valueIndex) ? -1 : Long.hashCode(b.getLong(firstValueIdx + valueIndex)); + } + + // TODO: those DateRange-specific sub-block getters, together with the AggregateMetricBuilder specific getters + // Should probably be refactored into some "composite block" interface, to avoid the need to implement all of + // them in ConstantArrayBlock. Something like `Block getSubBlock(int index)` would be enough. + // I think that was the original intent of CompositeBlock - not sure why it was abandoned. + LongBlock getFromBlock(); + + LongBlock getToBlock(); +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlockBuilder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlockBuilder.java new file mode 100644 index 0000000000000..0525d0cef4ec1 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlockBuilder.java @@ -0,0 +1,179 @@ +/* + * 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.compute.data; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.common.io.stream.GenericNamedWriteable; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.mapper.BlockLoader; + +import java.io.IOException; + +import static org.elasticsearch.index.mapper.RangeFieldMapper.ESQL_DATE_RANGE_CREATED_VERSION; + +public class DateRangeBlockBuilder extends AbstractBlockBuilder implements BlockLoader.DateRangeBuilder { + + private LongBlockBuilder fromBuilder; + private LongBlockBuilder toBuilder; + + public DateRangeBlockBuilder(int estimatedSize, BlockFactory blockFactory) { + super(blockFactory); + fromBuilder = null; + toBuilder = null; + try { + fromBuilder = new LongBlockBuilder(estimatedSize, blockFactory); + toBuilder = new LongBlockBuilder(estimatedSize, blockFactory); + } finally { + if (toBuilder == null) { + Releasables.closeWhileHandlingException(fromBuilder); + } + } + } + + @Override + protected int valuesLength() { + throw new UnsupportedOperationException("Not available on date_range"); + } + + @Override + protected void growValuesArray(int newSize) { + throw new UnsupportedOperationException("Not available on date_range"); + } + + @Override + protected int elementSize() { + throw new UnsupportedOperationException("Not available on date_range"); + } + + @Override + public long estimatedBytes() { + return fromBuilder.estimatedBytes() + toBuilder.estimatedBytes(); + } + + @Override + public DateRangeBlockBuilder copyFrom(Block b, int beginInclusive, int endExclusive) { + Block fromBlock; + Block toBlock; + if (b.areAllValuesNull()) { + fromBlock = b; + toBlock = b; + } else { + var block = (DateRangeArrayBlock) b; + fromBlock = block.getFromBlock(); + toBlock = block.getToBlock(); + } + fromBuilder.copyFrom(fromBlock, beginInclusive, endExclusive); + toBuilder.copyFrom(toBlock, beginInclusive, endExclusive); + return this; + } + + public DateRangeBlockBuilder copyFrom(DateRangeBlock block, int pos) { + if (block.isNull(pos)) { + appendNull(); + return this; + } + + if (block.getFromBlock().isNull(pos)) { + from().appendNull(); + } else { + from().appendLong(block.getFromBlock().getLong(pos)); + } + + if (block.getToBlock().isNull(pos)) { + to().appendNull(); + } else { + to().appendLong(block.getToBlock().getLong(pos)); + } + return this; + } + + @Override + public DateRangeBlockBuilder appendNull() { + fromBuilder.appendNull(); + toBuilder.appendNull(); + return this; + } + + public DateRangeBlockBuilder appendDateRange(DateRangeLiteral lit) { + fromBuilder.appendLong(lit.from); + toBuilder.appendLong(lit.to); + return this; + } + + @Override + public DateRangeBlockBuilder mvOrdering(Block.MvOrdering mvOrdering) { + fromBuilder.mvOrdering(mvOrdering); + toBuilder.mvOrdering(mvOrdering); + return this; + } + + @Override + public DateRangeBlock build() { + LongBlock fromBlock = null; + LongBlock toBlock = null; + boolean success = false; + try { + finish(); + fromBlock = fromBuilder.build(); + toBlock = toBuilder.build(); + var block = new DateRangeArrayBlock(fromBlock, toBlock); + success = true; + return block; + } finally { + if (success == false) { + Releasables.closeExpectNoException(fromBlock, toBlock); + } + } + } + + @Override + protected void extraClose() { + Releasables.closeExpectNoException(fromBuilder, toBuilder); + } + + @Override + public BlockLoader.LongBuilder from() { + return fromBuilder; + } + + @Override + public BlockLoader.LongBuilder to() { + return toBuilder; + } + + public record DateRangeLiteral(Long from, Long to) implements GenericNamedWriteable { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + GenericNamedWriteable.class, + "DateRangeLiteral", + DateRangeLiteral::new + ); + + public DateRangeLiteral(StreamInput in) throws IOException { + this(in.readOptionalLong(), in.readOptionalLong()); + } + + @Override + public String getWriteableName() { + return "DateRangeLiteral"; + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return ESQL_DATE_RANGE_CREATED_VERSION; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalLong(from); + out.writeOptionalLong(to); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ElementType.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ElementType.java index 5202231067787..5e7cf73edbe7f 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ElementType.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/ElementType.java @@ -64,7 +64,9 @@ public enum ElementType { "AggregateMetricDouble", BlockFactory::newAggregateMetricDoubleBlockBuilder, AggregateMetricDoubleArrayBlock::readFrom - ); + ), + + DATE_RANGE(11, "DateRange", BlockFactory::newDateRangeBlockBuilder, DateRangeArrayBlock::readFrom); private static final TransportVersion ESQL_SERIALIZE_BLOCK_TYPE_CODE = TransportVersion.fromName("esql_serialize_block_type_code"); @@ -113,6 +115,8 @@ public static ElementType fromJava(Class type) { elementType = BOOLEAN; } else if (type == AggregateMetricDoubleBlockBuilder.AggregateMetricDoubleLiteral.class) { elementType = AGGREGATE_METRIC_DOUBLE; + } else if (type == DateRangeBlockBuilder.DateRangeLiteral.class) { + elementType = DATE_RANGE; } else if (type == null || type == Void.class) { elementType = NULL; } else { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/read/DelegatingBlockLoaderFactory.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/read/DelegatingBlockLoaderFactory.java index 9e5f045b965ee..78e0ce9f9468e 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/read/DelegatingBlockLoaderFactory.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/read/DelegatingBlockLoaderFactory.java @@ -137,4 +137,9 @@ public BlockLoader.SortedSetOrdinalsBuilder sortedSetOrdinalsBuilder(SortedSetDo public BlockLoader.AggregateMetricDoubleBuilder aggregateMetricDoubleBuilder(int count) { return factory.newAggregateMetricDoubleBlockBuilder(count); } + + @Override + public BlockLoader.DateRangeBuilder dateRangeBuilder(int expectedCount) { + return factory.newDateRangeBlockBuilder(expectedCount); + } } 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 f300a400e60af..db86cd1d17157 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 @@ -188,6 +188,7 @@ public static IntFunction createBlockValueReader(Block block) { case DOC -> throw new IllegalArgumentException("can't read values from [doc] block"); case COMPOSITE -> throw new IllegalArgumentException("can't read values from [composite] block"); case AGGREGATE_METRIC_DOUBLE -> throw new IllegalArgumentException("can't read values from [aggregate metric double] block"); + case DATE_RANGE -> throw new IllegalArgumentException("can't read values from [date range] block"); case UNKNOWN -> throw new IllegalArgumentException("can't read values from [" + block + "]"); }; } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java index c97e3bf5fc223..8b99ce973e886 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java @@ -109,6 +109,7 @@ private void assertEvens(Block block) { (AggregateMetricDoubleBlock) block, i ); + case DATE_RANGE -> ((DateRangeBlockBuilder) builder).copyFrom((DateRangeBlock) block, i); default -> throw new IllegalArgumentException("unsupported type: " + elementType); } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java index 60c2d7dc0276c..018d3db1b0346 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java @@ -199,6 +199,9 @@ public void testCrankyConstantBlock() { } private void assumeMultiValued() { - assumeTrue("Type must support multi-values", elementType != ElementType.AGGREGATE_METRIC_DOUBLE); + assumeTrue( + "Type must support multi-values", + elementType != ElementType.AGGREGATE_METRIC_DOUBLE && elementType != ElementType.DATE_RANGE + ); } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java index f4872482af378..10e6f7251b22c 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/CompositeBlockTests.java @@ -21,7 +21,11 @@ public class CompositeBlockTests extends ComputeTestCase { static List supportedSubElementTypes = Arrays.stream(ElementType.values()) .filter( - e -> e != ElementType.COMPOSITE && e != ElementType.UNKNOWN && e != ElementType.DOC && e != ElementType.AGGREGATE_METRIC_DOUBLE + e -> e != ElementType.COMPOSITE + && e != ElementType.UNKNOWN + && e != ElementType.DOC + && e != ElementType.AGGREGATE_METRIC_DOUBLE + && e != ElementType.DATE_RANGE ) .toList(); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java index 5cb72177be6f7..9a8c00d2c15de 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java @@ -118,7 +118,7 @@ public void testCranky() { private Vector.Builder vectorBuilder(int estimatedSize, BlockFactory blockFactory) { return switch (elementType) { - case NULL, DOC, COMPOSITE, AGGREGATE_METRIC_DOUBLE, UNKNOWN -> throw new UnsupportedOperationException(); + case NULL, DOC, COMPOSITE, AGGREGATE_METRIC_DOUBLE, DATE_RANGE, UNKNOWN -> throw new UnsupportedOperationException(); case BOOLEAN -> blockFactory.newBooleanVectorBuilder(estimatedSize); case BYTES_REF -> blockFactory.newBytesRefVectorBuilder(estimatedSize); case FLOAT -> blockFactory.newFloatVectorBuilder(estimatedSize); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java index 0dbc177350f0b..944b57f5eeeec 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java @@ -118,7 +118,7 @@ public void testCranky() { private Vector.Builder vectorBuilder(int size, BlockFactory blockFactory) { return switch (elementType) { - case NULL, BYTES_REF, DOC, COMPOSITE, AGGREGATE_METRIC_DOUBLE, UNKNOWN -> throw new UnsupportedOperationException(); + case NULL, BYTES_REF, DOC, COMPOSITE, AGGREGATE_METRIC_DOUBLE, DATE_RANGE, UNKNOWN -> throw new UnsupportedOperationException(); case BOOLEAN -> blockFactory.newBooleanVectorFixedBuilder(size); case DOUBLE -> blockFactory.newDoubleVectorFixedBuilder(size); case FLOAT -> blockFactory.newFloatVectorFixedBuilder(size); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeTests.java index 4caa6415f0cfe..7dfebfebfa11b 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/mvdedupe/MultivalueDedupeTests.java @@ -64,7 +64,8 @@ public static List supportedTypes() { ElementType.DOC, ElementType.COMPOSITE, ElementType.FLOAT, - ElementType.AGGREGATE_METRIC_DOUBLE + ElementType.AGGREGATE_METRIC_DOUBLE, + ElementType.DATE_RANGE )) { continue; } diff --git a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java index a4c9e312eeb66..6f68b4f62a589 100644 --- a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java +++ b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java @@ -16,6 +16,8 @@ import org.elasticsearch.compute.data.BooleanBlock; import org.elasticsearch.compute.data.BytesRefBlock; import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DateRangeBlock; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.DocBlock; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.ElementType; @@ -71,6 +73,7 @@ public static Object randomValue(ElementType e) { ); case NULL -> null; case COMPOSITE -> throw new IllegalArgumentException("can't make random values for composite"); + case DATE_RANGE -> throw new IllegalArgumentException("can't make random values for date range"); case UNKNOWN -> throw new IllegalArgumentException("can't make random values for [" + e + "]"); }; } @@ -212,6 +215,11 @@ public static void append(Block.Builder builder, Object value) { b.count().appendInt(aggMetric.count()); return; } + if (builder instanceof DateRangeBlockBuilder b && value instanceof DateRangeBlockBuilder.DateRangeLiteral lit) { + b.from().appendLong(lit.from()); + b.to().appendLong(lit.to()); + return; + } if (builder instanceof DocBlock.Builder b && value instanceof BlockUtils.Doc v) { b.appendShard(v.shard()).appendSegment(v.segment()).appendDoc(v.doc()); return; @@ -301,6 +309,12 @@ public static List> valuesAtPositions(Block block, int from, int to yield literal; } + case DATE_RANGE -> { + var b = (DateRangeBlock) block; + var lit = new DateRangeBlockBuilder.DateRangeLiteral(b.getFromBlock().getLong(i), b.getToBlock().getLong(i)); + i++; + yield lit; + } default -> throw new IllegalArgumentException("unsupported element type [" + block.elementType() + "]"); }); } diff --git a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/RandomBlock.java b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/RandomBlock.java index b458f1f20254c..cd053e32e5774 100644 --- a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/RandomBlock.java +++ b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/RandomBlock.java @@ -13,6 +13,7 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.BooleanBlock; import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.FloatBlock; @@ -28,6 +29,10 @@ import java.util.List; import java.util.function.Supplier; +import static org.elasticsearch.common.time.DateUtils.MAX_MILLIS_BEFORE_9999; +import static org.elasticsearch.test.ESTestCase.randomLongBetween; +import static org.elasticsearch.test.ESTestCase.randomMillisUpToYear9999; + /** * A block of random values. * @param values the values as java object @@ -47,6 +52,7 @@ public static ElementType randomElementExcluding(List type) { || e == ElementType.NULL || e == ElementType.DOC || e == ElementType.COMPOSITE + || e == ElementType.DATE_RANGE || type.contains(e), () -> ESTestCase.randomFrom(ElementType.values()) ); @@ -155,6 +161,14 @@ public static RandomBlock randomBlock( b.count().appendInt(count); valuesAtPosition.add(new AggregateMetricDoubleBlockBuilder.AggregateMetricDoubleLiteral(min, max, sum, count)); } + case DATE_RANGE -> { + var b = (DateRangeBlockBuilder) builder; + var from = randomMillisUpToYear9999(); + var to = randomLongBetween(from + 1, MAX_MILLIS_BEFORE_9999); + b.from().appendLong(from); + b.to().appendLong(to); + valuesAtPosition.add(new DateRangeBlockBuilder.DateRangeLiteral(from, to)); + } default -> throw new IllegalArgumentException("unsupported element type [" + elementType + "]"); } } diff --git a/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java index 21418ec533663..4fe5bc74d2cc4 100644 --- a/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java +++ b/x-pack/plugin/esql/qa/server/single-node/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/single_node/RestEsqlIT.java @@ -764,6 +764,8 @@ public void testSuggestedCast() throws IOException { if (EsqlCapabilities.Cap.AGGREGATE_METRIC_DOUBLE_V0.isEnabled() == false) { shouldBeSupported.remove(DataType.AGGREGATE_METRIC_DOUBLE); } + // TODO this is temporary only until caps supppot is complete + shouldBeSupported.remove(DataType.DATE_RANGE); for (DataType type : shouldBeSupported) { assertTrue(type.typeName(), typesAndValues.containsKey(type)); } diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/AllSupportedFieldsTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/AllSupportedFieldsTestCase.java index 1e77a62711990..a68af6693d158 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/AllSupportedFieldsTestCase.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/AllSupportedFieldsTestCase.java @@ -398,6 +398,12 @@ private void createAllTypesDoc(RestClient client, String indexName) throws IOExc doc.field("value_count", 25); doc.endObject(); } + case DATE_RANGE -> { + doc.startObject(); + doc.field("gte", "1989-01-01"); + doc.field("lt", "2025-01-01"); + doc.endObject(); + } case DENSE_VECTOR -> doc.value(List.of(0.5, 10, 6)); default -> throw new AssertionError("unsupported field type [" + type + "]"); } @@ -438,6 +444,11 @@ private Matcher expectedValue(DataType type) { // TODO: Fix this once we know the node versions. yield nullValue(); } + case DATE_RANGE -> { + // Currently, we cannot tell if all nodes support it or not so we treat it as unsupported. + // TODO: Fix this once we know the node versions. + yield nullValue(); + } default -> throw new AssertionError("unsupported field type [" + type + "]"); }; } @@ -461,7 +472,9 @@ private static boolean supportedInIndex(DataType t) { // You can't index these - they are just constants. DATE_PERIOD, TIME_DURATION, GEOTILE, GEOHASH, GEOHEX, // TODO fix geo - CARTESIAN_POINT, CARTESIAN_SHAPE -> false; + CARTESIAN_POINT, CARTESIAN_SHAPE, + // Excluded for now, until a proper workaround + DATE_RANGE -> false; default -> true; }; } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java index 05d6b08b61ff2..f13642e7ba9ed 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.Page; import org.elasticsearch.geometry.utils.Geohash; import org.elasticsearch.h3.H3; @@ -19,6 +20,7 @@ import org.elasticsearch.search.aggregations.bucket.geogrid.GeoTileUtils; import org.elasticsearch.test.ListMatcher; import org.elasticsearch.xpack.esql.CsvTestUtils.ActualResults; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter; import org.elasticsearch.xpack.versionfield.Version; import org.hamcrest.Description; import org.hamcrest.Matchers; @@ -431,6 +433,11 @@ private static Object convertExpectedValue(Type expectedType, Object expectedVal AggregateMetricDoubleBlockBuilder.AggregateMetricDoubleLiteral.class, x -> aggregateMetricDoubleLiteralToString((AggregateMetricDoubleBlockBuilder.AggregateMetricDoubleLiteral) x) ); + case DATE_RANGE -> rebuildExpected( + expectedValue, + DateRangeBlockBuilder.DateRangeLiteral.class, + x -> EsqlDataTypeConverter.dateRangeLiteralToString((DateRangeBlockBuilder.DateRangeLiteral) x) + ); default -> expectedValue; }; } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestUtils.java index a6ce1b3b41b2e..b93bde18a3189 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestUtils.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestUtils.java @@ -20,6 +20,7 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.BlockUtils; import org.elasticsearch.compute.data.BlockUtils.BuilderWrapper; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.Page; import org.elasticsearch.core.Booleans; @@ -35,6 +36,7 @@ import org.elasticsearch.xpack.esql.action.ResponseValueUtils; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.util.StringUtils; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter; import org.supercsv.io.CsvListReader; import org.supercsv.prefs.CsvPreference; @@ -468,9 +470,7 @@ public enum Type { BytesRef.class ), IP_RANGE(InetAddresses::parseCidr, BytesRef.class), - INTEGER_RANGE(s -> s == null ? null : Arrays.stream(s.split("-")).map(Integer::parseInt).toArray(), int[].class), - DOUBLE_RANGE(s -> s == null ? null : Arrays.stream(s.split("-")).map(Double::parseDouble).toArray(), double[].class), - DATE_RANGE(s -> s == null ? null : Arrays.stream(s.split("-")).map(BytesRef::new).toArray(), BytesRef[].class), + DATE_RANGE(EsqlDataTypeConverter::dateRangeToLiteral, DateRangeBlockBuilder.DateRangeLiteral.class), VERSION(v -> new org.elasticsearch.xpack.versionfield.Version(v).toBytesRef(), BytesRef.class), NULL(s -> null, Void.class), DATETIME( @@ -593,6 +593,7 @@ public static Type asType(ElementType elementType, Type actualType) { case DOC -> throw new IllegalArgumentException("can't assert on doc blocks"); case COMPOSITE -> throw new IllegalArgumentException("can't assert on composite blocks"); case AGGREGATE_METRIC_DOUBLE -> AGGREGATE_METRIC_DOUBLE; + case DATE_RANGE -> DATE_RANGE; case UNKNOWN -> throw new IllegalArgumentException("Unknown block types cannot be handled"); }; } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java index 96f8ffdcccbbe..ffc9a92eca9e8 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java @@ -168,6 +168,7 @@ public class CsvTestsDataLoader { private static final TestDataset MV_TEXT = new TestDataset("mv_text"); private static final TestDataset DENSE_VECTOR = new TestDataset("dense_vector"); private static final TestDataset COLORS = new TestDataset("colors"); + private static final TestDataset DATE_RANGES = new TestDataset("date_ranges"); public static final Map CSV_DATASET_MAP = Map.ofEntries( Map.entry(EMPLOYEES.indexName, EMPLOYEES), @@ -234,7 +235,8 @@ public class CsvTestsDataLoader { Map.entry(DENSE_VECTOR.indexName, DENSE_VECTOR), Map.entry(COLORS.indexName, COLORS), Map.entry(MULTI_COLUMN_JOINABLE.indexName, MULTI_COLUMN_JOINABLE), - Map.entry(MULTI_COLUMN_JOINABLE_LOOKUP.indexName, MULTI_COLUMN_JOINABLE_LOOKUP) + Map.entry(MULTI_COLUMN_JOINABLE_LOOKUP.indexName, MULTI_COLUMN_JOINABLE_LOOKUP), + Map.entry(DATE_RANGES.indexName, DATE_RANGES) ); private static final EnrichConfig LANGUAGES_ENRICH = new EnrichConfig("languages_policy", "enrich-policy-languages.json"); diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java index 886f1b893fe80..1514d5814847b 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java @@ -34,6 +34,7 @@ import org.elasticsearch.compute.data.BlockFactoryProvider; import org.elasticsearch.compute.data.BlockUtils; import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.LongBlock; @@ -146,6 +147,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; +import static org.elasticsearch.common.time.DateUtils.MAX_MILLIS_BEFORE_9999; import static org.elasticsearch.test.ESTestCase.assertEquals; import static org.elasticsearch.test.ESTestCase.between; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; @@ -898,6 +900,11 @@ public static Literal randomLiteral(DataType type) { randomDouble(), randomInt() ); + case DATE_RANGE -> { + var from = randomMillisUpToYear9999(); + var to = randomLongBetween(from + 1, MAX_MILLIS_BEFORE_9999); + yield new DateRangeBlockBuilder.DateRangeLiteral(from, to); + } case NULL -> null; case SOURCE -> { try { diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/ages.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/ages.csv index a6637ffea55c9..5d240a73ca8d3 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/ages.csv +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/ages.csv @@ -1,11 +1,11 @@ age_range:integer_range, description:keyword - 0..2 , Baby - 2..4 , Toddler - 3..5 , Preschooler - 5..12 , Child - 13..20 , Adolescent - 20..40 , Young Adult - 40..60 , Middle-aged - 60..80 , Senior - 80..100, Elderly - 100..200, Incredible +0..2, Baby +2..4, Toddler +3..5, Preschooler +5..12, Child +13..20, Adolescent +20..40, Young Adult +40..60, Middle-aged +60..80, Senior +80..100, Elderly +100..200, Incredible diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/date_ranges.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/date_ranges.csv new file mode 100644 index 0000000000000..fc996306dc011 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/date_ranges.csv @@ -0,0 +1,14 @@ +date_range:date_range, decade:integer, description:keyword +1900-01-01..1910-01-01, 1900, Edwardian Era +1910-01-01..1920-01-01, 1910, Ragtime Era +1920-01-01..1930-01-01, 1920, Roaring Twenties +1930-01-01..1940-01-01, 1930, Dirty Thirties +1940-01-01..1950-01-01, 1940, Fabulous Forties +1950-01-01..1960-01-01, 1950, Nifty Fifties +1960-01-01..1970-01-01, 1960, Swinging Sixties +1970-01-01..1980-01-01, 1970, Groovy Seventies +1980-01-01..1990-01-01, 1980, Radical Eighties +1990-01-01..2000-01-01, 1990, Nineties Nostalgia +2000-01-01..2010-01-01, 2000, Innovation Explosion Decade +2010-01-01..2020-01-01, 2010, Renaissance Decade +2020-01-01..2030-01-01, 2020, Empowerment Era diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/decades.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/decades.csv index 487f07317c8c3..fc996306dc011 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/decades.csv +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/data/decades.csv @@ -1,14 +1,14 @@ - date_range:date_range, decade:integer, description:keyword -1900-01-01..1910-01-01, 1900, Edwardian Era -1910-01-01..1920-01-01, 1910, Ragtime Era -1920-01-01..1930-01-01, 1920, Roaring Twenties -1930-01-01..1940-01-01, 1930, Dirty Thirties -1940-01-01..1950-01-01, 1940, Fabulous Forties -1950-01-01..1960-01-01, 1950, Nifty Fifties -1960-01-01..1970-01-01, 1960, Swinging Sixties -1970-01-01..1980-01-01, 1970, Groovy Seventies -1980-01-01..1990-01-01, 1980, Radical Eighties -1990-01-01..2000-01-01, 1990, Nineties Nostalgia -2000-01-01..2010-01-01, 2000, Innovation Explosion Decade -2010-01-01..2020-01-01, 2010, Renaissance Decade -2020-01-01..2030-01-01, 2020, Empowerment Era +date_range:date_range, decade:integer, description:keyword +1900-01-01..1910-01-01, 1900, Edwardian Era +1910-01-01..1920-01-01, 1910, Ragtime Era +1920-01-01..1930-01-01, 1920, Roaring Twenties +1930-01-01..1940-01-01, 1930, Dirty Thirties +1940-01-01..1950-01-01, 1940, Fabulous Forties +1950-01-01..1960-01-01, 1950, Nifty Fifties +1960-01-01..1970-01-01, 1960, Swinging Sixties +1970-01-01..1980-01-01, 1970, Groovy Seventies +1980-01-01..1990-01-01, 1980, Radical Eighties +1990-01-01..2000-01-01, 1990, Nineties Nostalgia +2000-01-01..2010-01-01, 2000, Innovation Explosion Decade +2010-01-01..2020-01-01, 2010, Renaissance Decade +2020-01-01..2030-01-01, 2020, Empowerment Era diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date.csv-spec index e5efc9e7a37aa..d252391b06647 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date.csv-spec @@ -2028,3 +2028,24 @@ from employees emp_no:integer | birth_date:date | hire_date:date | monthName:keyword 10040 | null | 1993-02-14T00:00:00.000Z | null ; + +dateRange +required_capability: date_range_field_type + +from date_ranges; + +date_range:date_range | decade:integer | description:keyword +1900-01-01..1910-01-01 | 1900 | Edwardian Era +1910-01-01..1920-01-01 | 1910 | Ragtime Era +1920-01-01..1930-01-01 | 1920 | Roaring Twenties +1930-01-01..1940-01-01 | 1930 | Dirty Thirties +1940-01-01..1950-01-01 | 1940 | Fabulous Forties +1950-01-01..1960-01-01 | 1950 | Nifty Fifties +1960-01-01..1970-01-01 | 1960 | Swinging Sixties +1970-01-01..1980-01-01 | 1970 | Groovy Seventies +1980-01-01..1990-01-01 | 1980 | Radical Eighties +1990-01-01..2000-01-01 | 1990 | Nineties Nostalgia +2000-01-01..2010-01-01 | 2000 | Innovation Explosion Decade +2010-01-01..2020-01-01 | 2010 | Renaissance Decade +2020-01-01..2030-01-01 | 2020 | Empowerment Era +; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-date_ranges.json b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-date_ranges.json new file mode 100644 index 0000000000000..596d3975bd664 --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/mapping-date_ranges.json @@ -0,0 +1,13 @@ +{ + "properties": { + "date_range": { + "type": "date_range" + }, + "decade": { + "type": "integer" + }, + "description": { + "type": "keyword" + } + } + } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/PositionToXContent.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/PositionToXContent.java index 2e8a6d2941b4f..01412007d2bee 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/PositionToXContent.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/PositionToXContent.java @@ -14,6 +14,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BooleanBlock; import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DateRangeBlock; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.FloatBlock; import org.elasticsearch.compute.data.IntBlock; @@ -29,6 +30,7 @@ import static org.elasticsearch.xpack.esql.core.util.NumericUtils.unsignedLongAsNumber; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.aggregateMetricDoubleBlockToString; +import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateRangeToString; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateTimeToString; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.geoGridToString; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.ipToString; @@ -200,6 +202,15 @@ protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Pa return builder.value(((FloatBlock) block).getFloat(valueIndex)); } }; + case DATE_RANGE -> new PositionToXContent(block) { + @Override + protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) + throws IOException { + var from = ((DateRangeBlock) block).getFromBlock().getLong(valueIndex); + var to = ((DateRangeBlock) block).getToBlock().getLong(valueIndex); + return builder.value(dateRangeToString(from, to)); + } + }; case TSID_DATA_TYPE -> new PositionToXContent(block) { @Override protected XContentBuilder valueToXContent(XContentBuilder builder, ToXContent.Params params, int valueIndex) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/ResponseValueUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/ResponseValueUtils.java index e013836fde15f..929957598934a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/ResponseValueUtils.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/ResponseValueUtils.java @@ -15,6 +15,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BooleanBlock; import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DateRangeBlock; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.FloatBlock; import org.elasticsearch.compute.data.IntBlock; @@ -34,6 +35,7 @@ import static org.elasticsearch.xpack.esql.core.util.NumericUtils.unsignedLongAsNumber; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.aggregateMetricDoubleBlockToString; +import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateRangeToString; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateTimeToString; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.geoGridToString; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.ipToString; @@ -139,6 +141,11 @@ private static Object valueAt(DataType dataType, Block block, int offset, BytesR ); case GEOHEX, GEOHASH, GEOTILE -> geoGridToString(((LongBlock) block).getLong(offset), dataType); case AGGREGATE_METRIC_DOUBLE -> aggregateMetricDoubleBlockToString((AggregateMetricDoubleBlock) block, offset); + case DATE_RANGE -> { + var from = ((DateRangeBlock) block).getFromBlock().getLong(offset); + var to = ((DateRangeBlock) block).getToBlock().getLong(offset); + yield dateRangeToString(from, to); + } case UNSUPPORTED -> (String) null; case SOURCE -> { BytesRef val = ((BytesRefBlock) block).getBytesRef(offset, scratch); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java index 2488172cb184a..e62e9d216b7d5 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/PreAnalyzer.java @@ -29,9 +29,10 @@ public record PreAnalysis( List enriches, List lookupIndices, boolean supportsAggregateMetricDouble, - boolean supportsDenseVector + boolean supportsDenseVector, + boolean supportDateRange ) { - public static final PreAnalysis EMPTY = new PreAnalysis(null, null, List.of(), List.of(), false, false); + public static final PreAnalysis EMPTY = new PreAnalysis(null, null, List.of(), List.of(), false, false, false); } public PreAnalysis preAnalyze(LogicalPlan plan) { @@ -71,6 +72,7 @@ protected PreAnalysis doPreAnalyze(LogicalPlan plan) { */ Holder supportsAggregateMetricDouble = new Holder<>(false); Holder supportsDenseVector = new Holder<>(false); + Holder supportsDateRange = new Holder<>(false); plan.forEachDown(p -> p.forEachExpression(UnresolvedFunction.class, fn -> { if (fn.name().equalsIgnoreCase("knn") || fn.name().equalsIgnoreCase("to_dense_vector") @@ -96,7 +98,8 @@ protected PreAnalysis doPreAnalyze(LogicalPlan plan) { unresolvedEnriches, lookupIndices, indexMode.get() == IndexMode.TIME_SERIES || supportsAggregateMetricDouble.get(), - supportsDenseVector.get() + supportsDenseVector.get(), + supportsDateRange.get() ); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java index 0744fd126999d..95e93537a5ee9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java @@ -258,7 +258,7 @@ private Tuple mergeLookupResults( field.getTimeSeriesFieldType() ); EsField old = mappings.putIfAbsent(m.getKey(), field); - if (old != null && old.getDataType().equals(field.getDataType()) == false) { + if (old != null && typeMismatch(old.getDataType(), field.getDataType())) { String error = "field [" + m.getKey() + "] of enrich policy [" + policyName + "] has different data types "; error += "[" + old.getDataType() + ", " + field.getDataType() + "] across clusters"; return Tuple.tuple(null, error); @@ -284,6 +284,15 @@ private Tuple mergeLookupResults( return Tuple.tuple(resolved, null); } + // When trying to enrich with a field, check if there is a mismatch in the types. + // Apart from just comparing them, the only special case is DATE_RANGE: it used to be an enrichment-only + // field and then added as a first-class ESQL field type. So in enrichment over clusters it may appear as + // DATE_RANGE VS UNSUPPORTED, even though it's still the same type. + private boolean typeMismatch(DataType old, DataType field) { + if (old == DataType.DATE_RANGE && field == DataType.UNSUPPORTED) return false; + return old.equals(field) == false; + } + private String missingPolicyError(String policyName, Collection targetClusters, List missingClusters) { // local cluster only String reason = "cannot find enrich policy [" + policyName + "]"; @@ -455,6 +464,7 @@ public void messageReceived(LookupRequest request, TransportChannel channel, Tas // Disable aggregate_metric_double and dense_vector until we get version checks in planning false, false, + false, refs.acquire(indexResult -> { if (indexResult.isValid() && indexResult.get().concreteIndices().size() == 1) { EsIndex esIndex = indexResult.get(); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java index 7337d55b8f7df..d2dd38e751b60 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java @@ -129,10 +129,10 @@ public Nullability nullable() { protected TypeResolution resolveType() { return isType( field(), - dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR, + dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR && dt != DataType.DATE_RANGE, sourceText(), DEFAULT, - "any type except counter types or dense_vector" + "any type except counter types or dense_vector / date_range" ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java index 5ee5f371ca0fa..056b077aebf2e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java @@ -146,10 +146,10 @@ public Nullability nullable() { protected TypeResolution resolveType() { return isType( field(), - dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR, + dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR && dt != DataType.DATE_RANGE, sourceText(), DEFAULT, - "any type except counter types or dense_vector" + "any type except counter types or dense_vector / date_range" ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Present.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Present.java index c65601211b544..18b5d64727aa8 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Present.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Present.java @@ -135,10 +135,10 @@ public Nullability nullable() { protected TypeResolution resolveType() { return isType( field(), - dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR, + dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR && dt != DataType.DATE_RANGE, sourceText(), DEFAULT, - "any type except counter types or dense_vector" + "any type except counter types or dense_vector / date_range" ); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDateRange.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDateRange.java new file mode 100644 index 0000000000000..3a8b8ed1ac655 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToDateRange.java @@ -0,0 +1,74 @@ +/* + * 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.convert; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +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.core.type.DataType; +import org.elasticsearch.xpack.esql.expression.function.FunctionInfo; +import org.elasticsearch.xpack.esql.expression.function.Param; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class ToDateRange extends AbstractConvertFunction { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Expression.class, + "ToDateRange", + ToDateRange::new + ); + + private static final Map EVALUATORS = Map.ofEntries( + // TODO + ); + + @FunctionInfo(returnType = "DateRange", description = "TBD") + public ToDateRange( + Source source, + @Param( + name = "field", + type = {}, + description = "Input value. The input can be a single- or multi-valued column or an expression." + ) Expression field + ) { + super(source, field); + } + + private ToDateRange(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + @Override + protected Map factories() { + return EVALUATORS; + } + + @Override + public DataType dataType() { + return DataType.DATE_RANGE; + } + + @Override + public Expression replaceChildren(List newChildren) { + return new ToDateRange(source(), newChildren.getFirst()); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, ToDateRange::new, field()); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToString.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToString.java index 9b05e37774bb8..213b86ba51db8 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToString.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToString.java @@ -31,6 +31,7 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.CARTESIAN_SHAPE; import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_RANGE; import static org.elasticsearch.xpack.esql.core.type.DataType.DENSE_VECTOR; import static org.elasticsearch.xpack.esql.core.type.DataType.DOUBLE; import static org.elasticsearch.xpack.esql.core.type.DataType.GEOHASH; @@ -77,7 +78,8 @@ public class ToString extends AbstractConvertFunction implements EvaluatorMapper Map.entry(GEOHASH, (source, fieldEval) -> new ToStringFromGeoGridEvaluator.Factory(source, fieldEval, GEOHASH)), Map.entry(GEOTILE, (source, fieldEval) -> new ToStringFromGeoGridEvaluator.Factory(source, fieldEval, GEOTILE)), Map.entry(GEOHEX, (source, fieldEval) -> new ToStringFromGeoGridEvaluator.Factory(source, fieldEval, GEOHEX)), - Map.entry(AGGREGATE_METRIC_DOUBLE, ToStringFromAggregateMetricDoubleEvaluator.Factory::new) + Map.entry(AGGREGATE_METRIC_DOUBLE, ToStringFromAggregateMetricDoubleEvaluator.Factory::new), + Map.entry(DATE_RANGE, ToStringFromDateRangeEvaluator.Factory::new) ); @FunctionInfo( diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java new file mode 100644 index 0000000000000..de4f1fd161f86 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java @@ -0,0 +1,95 @@ +/* + * 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.convert; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.RamUsageEstimator; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DateRangeBlock; +import org.elasticsearch.compute.data.Vector; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.EvalOperator; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.xpack.esql.core.tree.Source; + +public class ToStringFromDateRangeEvaluator extends AbstractConvertFunction.AbstractEvaluator { + private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(ToStringFromDateRangeEvaluator.class); + + private final EvalOperator.ExpressionEvaluator field; + + public ToStringFromDateRangeEvaluator(Source source, EvalOperator.ExpressionEvaluator field, DriverContext driverContext) { + super(driverContext, source); + this.field = field; + } + + @Override + protected EvalOperator.ExpressionEvaluator next() { + return field; + } + + @Override + protected Block evalVector(Vector v) { + return evalBlock(v.asBlock()); + } + + private static BytesRef evalValue(DateRangeBlock block, int idx) { + return new BytesRef((block.getFromBlock().getLong(idx) + ".." + block.getToBlock().getLong(idx))); + } + + @Override + public Block evalBlock(Block b) { + var block = (DateRangeBlock) b; + int positionCount = block.getPositionCount(); + try (BytesRefBlock.Builder builder = driverContext.blockFactory().newBytesRefBlockBuilder(positionCount)) { + for (int p = 0; p < positionCount; p++) { + if (block.isNull(p)) { + builder.appendNull(); + } else { + builder.appendBytesRef(evalValue(block, p)); + } + } + return builder.build(); + } + } + + @Override + public String toString() { + return "ToStringFromDateRangeEvaluator[field=" + field + ']'; + } + + @Override + public long baseRamBytesUsed() { + return BASE_RAM_BYTES_USED + field.baseRamBytesUsed(); + } + + @Override + public void close() { + Releasables.closeExpectNoException(field); + } + + public static class Factory implements EvalOperator.ExpressionEvaluator.Factory { + private final Source source; + private final EvalOperator.ExpressionEvaluator.Factory field; + + public Factory(Source source, EvalOperator.ExpressionEvaluator.Factory field) { + this.source = source; + this.field = field; + } + + @Override + public EvalOperator.ExpressionEvaluator get(DriverContext context) { + return new ToStringFromDateRangeEvaluator(source, field.get(context), context); + } + + @Override + public String toString() { + return "ToStringFromDateRangeEvaluator[field=" + field + "]"; + } + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/nulls/Coalesce.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/nulls/Coalesce.java index 642eb27f7fb63..c1a891ada983a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/nulls/Coalesce.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/nulls/Coalesce.java @@ -209,7 +209,7 @@ public ExpressionEvaluator.Factory toEvaluator(ToEvaluator toEvaluator) { .toEvaluator(toEvaluator, children()); case NULL -> EvalOperator.CONSTANT_NULL_FACTORY; case UNSUPPORTED, SHORT, BYTE, DATE_PERIOD, OBJECT, DOC_DATA_TYPE, SOURCE, TIME_DURATION, FLOAT, HALF_FLOAT, TSID_DATA_TYPE, - SCALED_FLOAT, PARTIAL_AGG, AGGREGATE_METRIC_DOUBLE, DENSE_VECTOR -> throw new UnsupportedOperationException( + SCALED_FLOAT, PARTIAL_AGG, AGGREGATE_METRIC_DOUBLE, DENSE_VECTOR, DATE_RANGE -> throw new UnsupportedOperationException( dataType() + " can’t be coalesced" ); }; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AggregateMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AggregateMapper.java index 29edfc621e8d4..ec57e5bc83167 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AggregateMapper.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AggregateMapper.java @@ -111,7 +111,7 @@ private static DataType toDataType(ElementType elementType) { case INT -> DataType.INTEGER; case LONG -> DataType.LONG; case DOUBLE -> DataType.DOUBLE; - case FLOAT, NULL, DOC, COMPOSITE, AGGREGATE_METRIC_DOUBLE, UNKNOWN -> throw new EsqlIllegalArgumentException( + case FLOAT, NULL, DOC, COMPOSITE, AGGREGATE_METRIC_DOUBLE, DATE_RANGE, UNKNOWN -> throw new EsqlIllegalArgumentException( "unsupported agg type: " + elementType ); }; 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 91a5e40c0a1f5..b07f7db83b1b2 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 @@ -487,7 +487,7 @@ private PhysicalOperation planTopN(TopNExec topNExec, LocalExecutionPlannerConte case GEO_POINT, CARTESIAN_POINT, GEO_SHAPE, CARTESIAN_SHAPE, COUNTER_LONG, COUNTER_INTEGER, COUNTER_DOUBLE, SOURCE, AGGREGATE_METRIC_DOUBLE, DENSE_VECTOR, GEOHASH, GEOTILE, GEOHEX -> TopNEncoder.DEFAULT_UNSORTABLE; // unsupported fields are encoded as BytesRef, we'll use the same encoder; all values should be null at this point - case PARTIAL_AGG, UNSUPPORTED -> TopNEncoder.UNSUPPORTED; + case PARTIAL_AGG, UNSUPPORTED, DATE_RANGE -> TopNEncoder.UNSUPPORTED; }; } List orders = topNExec.order().stream().map(order -> { 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 c20034ad6e371..31407ea52242a 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 @@ -355,6 +355,7 @@ public static ElementType toElementType(DataType dataType, MappedFieldType.Field case PARTIAL_AGG -> ElementType.COMPOSITE; case AGGREGATE_METRIC_DOUBLE -> ElementType.AGGREGATE_METRIC_DOUBLE; case DENSE_VECTOR -> ElementType.FLOAT; + case DATE_RANGE -> ElementType.DATE_RANGE; case SHORT, BYTE, DATE_PERIOD, TIME_DURATION, OBJECT, FLOAT, HALF_FLOAT, SCALED_FLOAT -> throw EsqlIllegalArgumentException .illegalDataType(dataType); }; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index ded34107f5aec..2f3bdebfe54a5 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -513,6 +513,7 @@ private void preAnalyzeLookupIndex( // Disable aggregate_metric_double and dense_vector until we get version checks in planning false, false, + false, listener.map(indexResolution -> receiveLookupIndexResolution(result, localPattern, executionInfo, indexResolution)) ); } @@ -730,6 +731,7 @@ private void preAnalyzeMainIndices( preAnalysis.indexMode() == IndexMode.TIME_SERIES, preAnalysis.supportsAggregateMetricDouble(), preAnalysis.supportsDenseVector(), + preAnalysis.supportDateRange(), listener.delegateFailureAndWrap((l, indexResolution) -> { EsqlCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, indexResolution.failures()); l.onResponse(result.withIndices(indexResolution)); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/IndexResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/IndexResolver.java index 20936e6fa86f4..2d38d6605b088 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/IndexResolver.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/IndexResolver.java @@ -90,6 +90,7 @@ public void resolveAsMergedMapping( boolean includeAllDimensions, boolean supportsAggregateMetricDouble, boolean supportsDenseVector, + boolean supportsDateRange, ActionListener listener ) { client.execute( @@ -98,13 +99,21 @@ public void resolveAsMergedMapping( listener.delegateFailureAndWrap((l, response) -> { LOGGER.debug("minimum transport version {}", response.minTransportVersion()); l.onResponse( - mergedMappings(indexWildcard, new FieldsInfo(response.caps(), supportsAggregateMetricDouble, supportsDenseVector)) + mergedMappings( + indexWildcard, + new FieldsInfo(response.caps(), supportsAggregateMetricDouble, supportsDenseVector, supportsDateRange) + ) ); }) ); } - public record FieldsInfo(FieldCapabilitiesResponse caps, boolean supportAggregateMetricDouble, boolean supportDenseVector) {} + public record FieldsInfo( + FieldCapabilitiesResponse caps, + boolean supportAggregateMetricDouble, + boolean supportDenseVector, + boolean supportDateRange + ) {} // public for testing only public static IndexResolution mergedMappings(String indexPattern, FieldsInfo fieldsInfo) { @@ -242,6 +251,7 @@ private static EsField createField( type = switch (type) { case AGGREGATE_METRIC_DOUBLE -> fieldsInfo.supportAggregateMetricDouble ? AGGREGATE_METRIC_DOUBLE : UNSUPPORTED; case DENSE_VECTOR -> fieldsInfo.supportDenseVector ? DENSE_VECTOR : UNSUPPORTED; + case DATE_RANGE -> fieldsInfo.supportDateRange ? DATETIME : UNSUPPORTED; default -> type; }; boolean aggregatable = first.isAggregatable(); @@ -329,7 +339,6 @@ private static FieldCapabilitiesRequest createFieldCapsRequest( req.fields(fieldNames.toArray(String[]::new)); req.includeUnmapped(true); req.indexFilter(requestFilter); - req.returnLocalAll(false); // lenient because we throw our own errors looking at the response e.g. if something was not resolved // also because this way security doesn't throw authorization exceptions but rather honors ignore_unavailable req.indicesOptions(FIELD_CAPS_INDICES_OPTIONS); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java index 059cb20b8a08c..59d00df55f463 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java @@ -19,6 +19,7 @@ import org.elasticsearch.compute.data.AggregateMetricDoubleBlock; import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder; import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder.Metric; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.core.Booleans; @@ -46,6 +47,7 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToCartesianShape; import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDateNanos; import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDatePeriod; +import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDateRange; import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDatetime; import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDenseVector; import org.elasticsearch.xpack.esql.expression.function.scalar.convert.ToDouble; @@ -92,6 +94,7 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_PERIOD; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_RANGE; import static org.elasticsearch.xpack.esql.core.type.DataType.DENSE_VECTOR; import static org.elasticsearch.xpack.esql.core.type.DataType.DOUBLE; import static org.elasticsearch.xpack.esql.core.type.DataType.GEOHASH; @@ -142,6 +145,7 @@ public class EsqlDataTypeConverter { typeToConverter.put(CARTESIAN_SHAPE, ToCartesianShape::new); typeToConverter.put(DATETIME, ToDatetime::new); typeToConverter.put(DATE_NANOS, ToDateNanos::new); + typeToConverter.put(DATE_RANGE, ToDateRange::new); // ToDegrees, typeless typeToConverter.put(DENSE_VECTOR, ToDenseVector::new); typeToConverter.put(DOUBLE, ToDouble::new); @@ -232,6 +236,8 @@ public enum INTERVALS { INTERVALS.Y ); + public record Range(T from, T to) {} + public static final String INVALID_INTERVAL_ERROR = "Invalid interval value in [{}], expected integer followed by one of {} but got [{}]"; @@ -652,6 +658,25 @@ public static String nanoTimeToString(long dateTime, DateFormatter formatter) { return formatter == null ? nanoTimeToString(dateTime) : formatter.formatNanos(dateTime); } + public static Range dateRangeToLongs(String s) { + var ss = s.split("\\.\\."); + assert ss.length == 2 : "can't parse range: " + s; + return new Range<>(dateTimeToLong(ss[0]), dateTimeToLong(ss[1]) - 1); + } + + public static DateRangeBlockBuilder.DateRangeLiteral dateRangeToLiteral(String s) { + var range = dateRangeToLongs(s); + return new DateRangeBlockBuilder.DateRangeLiteral(range.from, range.to); + } + + public static String dateRangeLiteralToString(DateRangeBlockBuilder.DateRangeLiteral lit) { + return dateRangeToString(lit.from(), lit.to()); + } + + public static String dateRangeToString(long from, long to) { + return dateTimeToString(from) + ".." + dateTimeToString(to); + } + public static BytesRef numericBooleanToString(Object field) { return new BytesRef(String.valueOf(field)); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/SerializationTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/SerializationTestUtils.java index e55a1b039258e..234c5ed5ebc0b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/SerializationTestUtils.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/SerializationTestUtils.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.ExistsQueryBuilder; import org.elasticsearch.index.query.MatchAllQueryBuilder; @@ -123,6 +124,13 @@ public static NamedWriteableRegistry writableRegistry() { AggregateMetricDoubleBlockBuilder.AggregateMetricDoubleLiteral::new ) ); + entries.add( + new NamedWriteableRegistry.Entry( + GenericNamedWriteable.class, + DateRangeBlockBuilder.DateRangeLiteral.ENTRY.name, + DateRangeBlockBuilder.DateRangeLiteral::new + ) + ); return new NamedWriteableRegistry(entries); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java index 77f73f430868e..138ea56435775 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java @@ -84,6 +84,7 @@ import static org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes.CARTESIAN; import static org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes.GEO; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateNanosToLong; +import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateRangeToLongs; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.dateTimeToLong; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.longToUnsignedLong; import static org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter.stringToIP; @@ -1262,6 +1263,12 @@ static Page valuesToPage(BlockFactory blockFactory, List columns byte[] decode = Base64.getUrlDecoder().decode(value.toString()); ((BytesRefBlock.Builder) builder).appendBytesRef(new BytesRef(decode)); } + case DATE_RANGE -> { + BlockLoader.DateRangeBuilder b = (BlockLoader.DateRangeBuilder) builder; + var ll = dateRangeToLongs(value.toString()); + b.from().appendLong(ll.from()); + b.to().appendLong(ll.to()); + } } } } 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 1a096df1b0648..0fea25dac6e55 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 @@ -3160,6 +3160,7 @@ public void testResolveInsist_multiIndexFieldPartiallyMappedWithSingleKeywordTyp List.of() ), true, + true, true ) ); @@ -3187,6 +3188,7 @@ public void testResolveInsist_multiIndexFieldExistsWithSingleTypeButIsNotKeyword List.of() ), true, + true, true ) ); @@ -3216,6 +3218,7 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesNoKeyw List.of() ), true, + true, true ) ); @@ -3243,6 +3246,7 @@ public void testResolveInsist_multiIndexSameMapping_fieldIsMapped() { List.of() ), true, + true, true ) ); @@ -3270,6 +3274,7 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithKe List.of() ), true, + true, true ) ); @@ -3298,6 +3303,7 @@ public void testResolveInsist_multiIndexFieldPartiallyExistsWithMultiTypesWithCa List.of() ), true, + true, true ) ); @@ -3318,13 +3324,13 @@ public void testResolveDenseVector() { List.of() ); { - IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true)); + IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true, true)); var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER)); assertThat(plan.output(), hasSize(1)); assertThat(plan.output().getFirst().dataType(), equalTo(DENSE_VECTOR)); } { - IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, false)); + IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, false, true)); var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER)); assertThat(plan.output(), hasSize(1)); assertThat(plan.output().getFirst().dataType(), equalTo(UNSUPPORTED)); @@ -3342,7 +3348,7 @@ public void testResolveAggregateMetricDouble() { List.of() ); { - IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true)); + IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, true, true, true)); var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER)); assertThat(plan.output(), hasSize(1)); assertThat( @@ -3351,7 +3357,7 @@ public void testResolveAggregateMetricDouble() { ); } { - IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, false, true)); + IndexResolution resolution = IndexResolver.mergedMappings("foo", new IndexResolver.FieldsInfo(caps, false, true, true)); var plan = analyze("FROM foo", analyzer(resolution, TEST_VERIFIER)); assertThat(plan.output(), hasSize(1)); assertThat(plan.output().getFirst().dataType(), equalTo(UNSUPPORTED)); @@ -3801,7 +3807,12 @@ private static LogicalPlan analyzeWithEmptyFieldCapsResponse(String query) throw List idxResponses = List.of( new FieldCapabilitiesIndexResponse("idx", "idx", Map.of(), true, IndexMode.STANDARD) ); - IndexResolver.FieldsInfo caps = new IndexResolver.FieldsInfo(new FieldCapabilitiesResponse(idxResponses, List.of()), true, true); + IndexResolver.FieldsInfo caps = new IndexResolver.FieldsInfo( + new FieldCapabilitiesResponse(idxResponses, List.of()), + true, + true, + true + ); IndexResolution resolution = IndexResolver.mergedMappings("test*", caps); var analyzer = analyzer(resolution, TEST_VERIFIER, configuration(query)); return analyze(query, analyzer); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/TestCaseSupplier.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/TestCaseSupplier.java index 3d75db67e7604..be6fce03cae9d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/TestCaseSupplier.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/TestCaseSupplier.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.network.InetAddresses; import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.geo.GeometryTestUtils; import org.elasticsearch.geo.ShapeTestUtils; import org.elasticsearch.geometry.Point; @@ -54,9 +55,12 @@ import java.util.function.UnaryOperator; import java.util.stream.Collectors; +import static org.elasticsearch.common.time.DateUtils.MAX_MILLIS_BEFORE_9999; import static org.elasticsearch.test.ESTestCase.randomDouble; import static org.elasticsearch.test.ESTestCase.randomFloatBetween; import static org.elasticsearch.test.ESTestCase.randomIntBetween; +import static org.elasticsearch.test.ESTestCase.randomLongBetween; +import static org.elasticsearch.test.ESTestCase.randomMillisUpToYear9999; import static org.elasticsearch.test.ESTestCase.randomNonNegativeInt; import static org.elasticsearch.xpack.esql.core.util.NumericUtils.UNSIGNED_LONG_MAX; import static org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes.CARTESIAN; @@ -868,6 +872,23 @@ public static void forUnaryAggregateMetricDouble( ); } + public static void forUnaryDateRange( + List suppliers, + String expectedEvaluatorToString, + DataType expectedType, + Function expectedValue, + List warnings + ) { + unary( + suppliers, + expectedEvaluatorToString, + dateRangeCases(), + expectedType, + v -> expectedValue.apply((DateRangeBlockBuilder.DateRangeLiteral) v), + warnings + ); + } + private static void unaryNumeric( List suppliers, String expectedEvaluatorToString, @@ -1506,6 +1527,16 @@ public static List aggregateMetricDoubleCases() { ); } + public static List dateRangeCases() { + return List.of(new TypedDataSupplier("", TestCaseSupplier::randomDateRange, DataType.DATE_RANGE)); + } + + static DateRangeBlockBuilder.DateRangeLiteral randomDateRange() { + var from = randomMillisUpToYear9999(); + var to = randomLongBetween(from + 1, MAX_MILLIS_BEFORE_9999); + return new DateRangeBlockBuilder.DateRangeLiteral(from, to); + } + public static String getCastEvaluator(String original, DataType current, DataType target) { if (current == target) { return original; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringTests.java index afba5b43878e0..fb7c06bfe99e0 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringTests.java @@ -169,6 +169,15 @@ public static Iterable parameters() { agg -> new BytesRef(EsqlDataTypeConverter.aggregateMetricDoubleLiteralToString(agg)), List.of() ); + + TestCaseSupplier.forUnaryDateRange( + suppliers, + "ToStringFromDateRangeEvaluator[field=" + read + "]", + DataType.KEYWORD, + dr -> new BytesRef(EsqlDataTypeConverter.dateRangeLiteralToString(dr)), + List.of() + ); + return parameterSuppliersFromTypedDataWithDefaultChecksNoErrors(true, suppliers); } 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 219b0a3586a6c..dd476b5207c0c 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 @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.search.SearchModule; import org.elasticsearch.xpack.esql.core.tree.Node; import org.elasticsearch.xpack.esql.expression.ExpressionWritables; @@ -55,6 +56,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(DateRangeBlockBuilder.DateRangeLiteral.ENTRY); entries.add(LookupJoinExec.ENTRY); return new NamedWriteableRegistry(entries); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java index fbb22c49af331..700047140c91c 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeRegistryTests.java @@ -54,7 +54,7 @@ private void resolve(String esTypeName, TimeSeriesParams.MetricType metricType, FieldCapabilitiesResponse caps = new FieldCapabilitiesResponse(idxResponses, List.of()); // IndexResolver uses EsqlDataTypeRegistry directly - IndexResolution resolution = IndexResolver.mergedMappings("idx-*", new IndexResolver.FieldsInfo(caps, true, true)); + IndexResolution resolution = IndexResolver.mergedMappings("idx-*", new IndexResolver.FieldsInfo(caps, true, true, true)); EsField f = resolution.get().mapping().get(field); assertThat(f.getDataType(), equalTo(expected)); } From e971c44d8b860264d95ad2328712f19bb30079be Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Wed, 8 Oct 2025 23:23:40 +0000 Subject: [PATCH 2/8] [CI] Update transport version definitions --- .../definitions/referable/esql_date_range_created_version.csv | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv b/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv index ce3efd6a04801..d617a6ff2f5d2 100644 --- a/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv +++ b/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv @@ -1 +1 @@ -9190000 +9191000 From af95bad94d862ff4c785490b6887af028fe8e047 Mon Sep 17 00:00:00 2001 From: shmuelhanoch Date: Thu, 9 Oct 2025 02:31:38 +0300 Subject: [PATCH 3/8] Update docs/changelog/133309.yaml --- docs/changelog/133309.yaml | 5 +++++ .../xpack/esql/core/expression/TypeResolutions.java | 13 +++++++++---- .../elasticsearch/compute/data/DateRangeBlock.java | 8 ++++++-- .../xpack/esql/enrich/EnrichPolicyResolver.java | 4 +++- .../esql/expression/function/aggregate/Absent.java | 2 +- .../esql/expression/function/aggregate/Count.java | 2 +- .../convert/ToStringFromDateRangeEvaluator.java | 7 ++++++- .../xpack/esql/action/EsqlQueryResponseTests.java | 3 ++- .../xpack/esql/analysis/VerifierTests.java | 2 +- .../function/AbstractAggregationTestCase.java | 1 + .../function/aggregate/AbsentErrorTests.java | 8 ++++++-- .../function/aggregate/CountErrorTests.java | 8 ++++++-- .../function/aggregate/PresentErrorTests.java | 6 +++++- .../function/aggregate/SampleErrorTests.java | 8 +++++--- .../function/aggregate/ValuesErrorTests.java | 2 +- .../scalar/multivalue/MvAppendErrorTests.java | 2 +- .../scalar/multivalue/MvContainsErrorTests.java | 4 ++-- .../scalar/multivalue/MvCountErrorTests.java | 4 ++-- .../scalar/multivalue/MvDedupeErrorTests.java | 2 +- .../scalar/multivalue/MvFirstErrorTests.java | 4 ++-- .../scalar/multivalue/MvLastErrorTests.java | 4 ++-- .../function/scalar/multivalue/MvMaxErrorTests.java | 2 +- .../function/scalar/multivalue/MvMinErrorTests.java | 2 +- .../scalar/multivalue/MvSliceErrorTests.java | 2 +- 24 files changed, 71 insertions(+), 34 deletions(-) create mode 100644 docs/changelog/133309.yaml diff --git a/docs/changelog/133309.yaml b/docs/changelog/133309.yaml new file mode 100644 index 0000000000000..80fa837eb9f05 --- /dev/null +++ b/docs/changelog/133309.yaml @@ -0,0 +1,5 @@ +pr: 133309 +summary: support DATE_RANGE field type +area: ES|QL +type: enhancement +issues: [] diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/TypeResolutions.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/TypeResolutions.java index 56bbf69c76e55..6a20478028d85 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/TypeResolutions.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/TypeResolutions.java @@ -21,6 +21,7 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.AGGREGATE_METRIC_DOUBLE; import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_RANGE; import static org.elasticsearch.xpack.esql.core.type.DataType.DENSE_VECTOR; import static org.elasticsearch.xpack.esql.core.type.DataType.IP; import static org.elasticsearch.xpack.esql.core.type.DataType.NULL; @@ -85,10 +86,10 @@ public static TypeResolution isRepresentableExceptCountersDenseVectorAndAggregat ) { return isType( e, - dt -> isRepresentable(dt) && dt != DENSE_VECTOR && dt != AGGREGATE_METRIC_DOUBLE, + dt -> isRepresentable(dt) && dt != DENSE_VECTOR && dt != AGGREGATE_METRIC_DOUBLE && dt != DATE_RANGE, operationName, paramOrd, - "any type except counter types, dense_vector, or aggregate_metric_double" + "any type except counter types, dense_vector, aggregate_metric_double or date_range" ); } @@ -99,10 +100,14 @@ public static TypeResolution isRepresentableExceptCountersSpatialDenseVectorAndA ) { return isType( e, - (t) -> isSpatialOrGrid(t) == false && DataType.isRepresentable(t) && t != DENSE_VECTOR && t != AGGREGATE_METRIC_DOUBLE, + (t) -> isSpatialOrGrid(t) == false + && DataType.isRepresentable(t) + && t != DENSE_VECTOR + && t != AGGREGATE_METRIC_DOUBLE + && t != DATE_RANGE, operationName, paramOrd, - "any type except counter, spatial types, dense_vector, or aggregate_metric_double" + "any type except counter, spatial types, dense_vector, aggregate_metric_double or date_range" ); } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java index d8f93f940b976..58fb2902e4c2a 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DateRangeBlock.java @@ -33,8 +33,12 @@ public sealed interface DateRangeBlock extends Block permits DateRangeArrayBlock * equals method works properly across different implementations of the AggregateMetricDoubleBlock interface. */ static boolean equals(DateRangeBlock lhs, DateRangeBlock rhs) { - if (lhs == rhs) return true; - if (lhs.getPositionCount() != rhs.getPositionCount()) return false; + if (lhs == rhs) { + return true; + } + if (lhs.getPositionCount() != rhs.getPositionCount()) { + return false; + } return LongBlock.equals(lhs.getFromBlock(), rhs.getFromBlock()) && LongBlock.equals(lhs.getToBlock(), rhs.getToBlock()); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java index 95e93537a5ee9..952bc840788a7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichPolicyResolver.java @@ -289,7 +289,9 @@ private Tuple mergeLookupResults( // field and then added as a first-class ESQL field type. So in enrichment over clusters it may appear as // DATE_RANGE VS UNSUPPORTED, even though it's still the same type. private boolean typeMismatch(DataType old, DataType field) { - if (old == DataType.DATE_RANGE && field == DataType.UNSUPPORTED) return false; + if (old == DataType.DATE_RANGE && field == DataType.UNSUPPORTED) { + return false; + } return old.equals(field) == false; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java index d2dd38e751b60..c00c2c063acb9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Absent.java @@ -132,7 +132,7 @@ protected TypeResolution resolveType() { dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR && dt != DataType.DATE_RANGE, sourceText(), DEFAULT, - "any type except counter types or dense_vector / date_range" + "any type except counter types, dense_vector or date_range" ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java index 056b077aebf2e..523b2265b15ab 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/aggregate/Count.java @@ -149,7 +149,7 @@ protected TypeResolution resolveType() { dt -> dt.isCounter() == false && dt != DataType.DENSE_VECTOR && dt != DataType.DATE_RANGE, sourceText(), DEFAULT, - "any type except counter types or dense_vector / date_range" + "any type except counter types, dense_vector or date_range" ); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java index de4f1fd161f86..9c51340e61c97 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToStringFromDateRangeEvaluator.java @@ -17,6 +17,7 @@ import org.elasticsearch.compute.operator.EvalOperator; import org.elasticsearch.core.Releasables; import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter; public class ToStringFromDateRangeEvaluator extends AbstractConvertFunction.AbstractEvaluator { private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(ToStringFromDateRangeEvaluator.class); @@ -39,7 +40,11 @@ protected Block evalVector(Vector v) { } private static BytesRef evalValue(DateRangeBlock block, int idx) { - return new BytesRef((block.getFromBlock().getLong(idx) + ".." + block.getToBlock().getLong(idx))); + return new BytesRef( + (EsqlDataTypeConverter.dateTimeToString(block.getFromBlock().getLong(idx)) + + ".." + + EsqlDataTypeConverter.dateTimeToString(block.getToBlock().getLong(idx))) + ); } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java index 138ea56435775..3f23665f47d29 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java @@ -196,7 +196,8 @@ private ColumnInfoImpl randomColumnInfo() { || t == DataType.TIME_DURATION || t == DataType.PARTIAL_AGG || t == DataType.AGGREGATE_METRIC_DOUBLE - || t == DataType.TSID_DATA_TYPE, + || t == DataType.TSID_DATA_TYPE + || t == DataType.DATE_RANGE, () -> randomFrom(DataType.types()) ).widenSmallNumeric(); return new ColumnInfoImpl(randomAlphaOfLength(10), type.esType(), randomOriginalTypes()); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 389fadfed1693..5b086a0a27d1d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1134,7 +1134,7 @@ public void testAggregateOnCounter() { assertThat( error("FROM tests | STATS count(network.bytes_out)", tsdb), equalTo( - "1:20: argument of [count(network.bytes_out)] must be [any type except counter types or dense_vector]," + "1:20: argument of [count(network.bytes_out)] must be [any type except counter types, dense_vector or date_range]," + " found value [network.bytes_out] type [counter_long]" ) ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractAggregationTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractAggregationTestCase.java index 69b6ff4ff30d1..7577cb613487a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractAggregationTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractAggregationTestCase.java @@ -560,6 +560,7 @@ protected static String standardAggregatorName(String prefix, DataType type) { case IP -> "Ip"; case DATETIME, DATE_NANOS, LONG, COUNTER_LONG, UNSIGNED_LONG, GEOHASH, GEOTILE, GEOHEX -> "Long"; case AGGREGATE_METRIC_DOUBLE -> "AggregateMetricDouble"; + case DATE_RANGE -> "DateRange"; case NULL -> "Null"; default -> throw new UnsupportedOperationException("name for [" + type + "]"); }; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/AbsentErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/AbsentErrorTests.java index 0e19d0c49953d..531b6f17314c9 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/AbsentErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/AbsentErrorTests.java @@ -33,12 +33,16 @@ protected Expression build(Source source, List args) { @Override protected Matcher expectedTypeErrorMatcher(List> validPerPosition, List signature) { return equalTo( - "argument of [dense_vector] must be [any type except counter types or dense_vector], found value [] type [dense_vector]" + "argument of [" + + signature.getFirst().esType() + + "] must be [any type except counter types, dense_vector or date_range], found value [] type [" + + signature.getFirst().esType() + + "]" ); } @Override protected void assertNumberOfCheckedSignatures(int checked) { - assertThat(checked, equalTo(1)); + assertThat(checked, equalTo(2)); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/CountErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/CountErrorTests.java index c66c4aa9aaeec..4aa198493118a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/CountErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/CountErrorTests.java @@ -33,12 +33,16 @@ protected Expression build(Source source, List args) { @Override protected Matcher expectedTypeErrorMatcher(List> validPerPosition, List signature) { return equalTo( - "argument of [dense_vector] must be [any type except counter types or dense_vector], found value [] type [dense_vector]" + "argument of [" + + signature.getFirst().esType() + + "] must be [any type except counter types, dense_vector or date_range], found value [] type [" + + signature.getFirst().esType() + + "]" ); } @Override protected void assertNumberOfCheckedSignatures(int checked) { - assertThat(checked, equalTo(1)); + assertThat(checked, equalTo(2)); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/PresentErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/PresentErrorTests.java index 0be345a9799e3..a5d839ccb135a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/PresentErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/PresentErrorTests.java @@ -33,7 +33,11 @@ protected Expression build(Source source, List args) { @Override protected Matcher expectedTypeErrorMatcher(List> validPerPosition, List signature) { return equalTo( - "argument of [dense_vector] must be [any type except counter types or dense_vector], found value [] type [dense_vector]" + "argument of [" + + signature.getFirst().esType() + + "] must be [any type except counter types, dense_vector or date_range], found value [] type [" + + signature.getFirst().esType() + + "]" ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/SampleErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/SampleErrorTests.java index 70cd9dfd810ca..ce71ca5d269a8 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/SampleErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/SampleErrorTests.java @@ -33,12 +33,14 @@ protected Expression build(Source source, List args) { @Override protected Matcher expectedTypeErrorMatcher(List> validPerPosition, List signature) { - if (signature.getFirst() == DataType.DENSE_VECTOR) { + if (signature.getFirst() == DataType.DENSE_VECTOR || signature.getFirst() == DataType.DATE_RANGE) { return equalTo( "first argument of [" + sourceForSignature(signature) + "] must be [any type except counter types, dense_vector," - + " or aggregate_metric_double], found value [] type [dense_vector]" + + " aggregate_metric_double, dense_vector or date_range], found value [] type [" + + signature.getFirst() + + "]" ); } if (signature.getFirst() == DataType.NULL && signature.get(1) == DataType.DENSE_VECTOR) { @@ -52,7 +54,7 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP true, validPerPosition, signature, - (v, p) -> p == 1 ? "integer" : "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> p == 1 ? "integer" : "any type except counter types, dense_vector, aggregate_metric_double or date_range" ) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/ValuesErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/ValuesErrorTests.java index b8a68786e9913..03a7512d57899 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/ValuesErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/aggregate/ValuesErrorTests.java @@ -37,7 +37,7 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter types, dense_vector, aggregate_metric_double or date_range" ) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAppendErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAppendErrorTests.java index 22e6a496aced8..7c29b3931d041 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAppendErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvAppendErrorTests.java @@ -41,7 +41,7 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter types, dense_vector, aggregate_metric_double or date_range" ) ); } else { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvContainsErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvContainsErrorTests.java index c34a7638ef944..1a4151bf9f33a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvContainsErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvContainsErrorTests.java @@ -33,7 +33,7 @@ protected Expression build(Source source, List args) { @Override protected Matcher expectedTypeErrorMatcher(List> validPerPosition, List signature) { - var unsupportedTypes = List.of(DataType.AGGREGATE_METRIC_DOUBLE, DataType.DENSE_VECTOR); + var unsupportedTypes = List.of(DataType.AGGREGATE_METRIC_DOUBLE, DataType.DENSE_VECTOR, DataType.DATE_RANGE); if (unsupportedTypes.contains(signature.getFirst()) || signature.getFirst() == DataType.NULL && unsupportedTypes.contains(signature.get(1))) { return containsString( @@ -41,7 +41,7 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter types, dense_vector, aggregate_metric_double or date_range" ) ); } else { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCountErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCountErrorTests.java index 60814dc70b3e8..9fe4a58ae4012 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCountErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvCountErrorTests.java @@ -37,14 +37,14 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter types, dense_vector, aggregate_metric_double, or date_range" ) ); } @Override protected void assertNumberOfCheckedSignatures(int checked) { - assertThat("all signatures except dense_vector and aggregate_metric_double should be supported", checked, equalTo(2)); + assertThat("all signatures except dense_vector, aggregate_metric_double or date_range should be supported", checked, equalTo(2)); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupeErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupeErrorTests.java index de367326fbf74..8f73527ebde4b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupeErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvDedupeErrorTests.java @@ -37,7 +37,7 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter types, dense_vector, aggregate_metric_double or date_range" ) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvFirstErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvFirstErrorTests.java index c0a707cdb41d4..0dbe12fb624c5 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvFirstErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvFirstErrorTests.java @@ -37,13 +37,13 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter types, dense_vector, aggregate_metric_double or date_range" ) ); } @Override protected void assertNumberOfCheckedSignatures(int checked) { - assertThat("all signatures except dense_vector and aggregate_metric_double should be supported", checked, equalTo(2)); + assertThat("all signatures except dense_vector, aggregate_metric_double or date_range should be supported", checked, equalTo(3)); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvLastErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvLastErrorTests.java index 6ad79c8d698f8..a1992de0b6278 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvLastErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvLastErrorTests.java @@ -37,13 +37,13 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter types, dense_vector, aggregate_metric_double or date_range" ) ); } @Override protected void assertNumberOfCheckedSignatures(int checked) { - assertThat("all signatures except dense_vector and aggregate_metric_double should be supported", checked, equalTo(2)); + assertThat("all signatures except dense_vector, aggregate_metric_double or date_range should be supported", checked, equalTo(2)); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxErrorTests.java index a31c00e4f7590..f9f541ac947dc 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMaxErrorTests.java @@ -37,7 +37,7 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter, spatial types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter, spatial types, dense_vector, aggregate_metric_double or date_range" ) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinErrorTests.java index 46009bba850b9..2b30fad1a1e07 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvMinErrorTests.java @@ -37,7 +37,7 @@ protected Matcher expectedTypeErrorMatcher(List> validPerP false, validPerPosition, signature, - (v, p) -> "any type except counter, spatial types, dense_vector, or aggregate_metric_double" + (v, p) -> "any type except counter, spatial types, dense_vector, aggregate_metric_double or date_range" ) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSliceErrorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSliceErrorTests.java index 49cefcd9adbd4..5974803274adc 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSliceErrorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvSliceErrorTests.java @@ -34,7 +34,7 @@ protected Expression build(Source source, List args) { protected Matcher expectedTypeErrorMatcher(List> validPerPosition, List signature) { return equalTo(typeErrorMessage(true, validPerPosition, signature, (v, p) -> switch (p) { case 1, 2 -> "integer"; - default -> "any type except counter types, dense_vector, or aggregate_metric_double"; + default -> "any type except counter types, dense_vector, aggregate_metric_double or date_range"; })); } } From 22dbda4f068e48645fe62b7cfa70d8785e8ac3fe Mon Sep 17 00:00:00 2001 From: Shmuel Hanoch Date: Fri, 10 Oct 2025 00:02:38 +0300 Subject: [PATCH 4/8] Suppres two more tests --- .../java/org/elasticsearch/compute/data/BlockBuilderTests.java | 2 +- .../elasticsearch/compute/data/VectorFixedBuilderTests.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java index 018d3db1b0346..0661af5ecb644 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderTests.java @@ -183,7 +183,7 @@ public void testCrankyConstantBlock() { RandomBlock random = RandomBlock.randomBlock(elementType, 1, false, 1, 1, 0, 0); builder.copyFrom(random.block(), 0, random.block().getPositionCount()); try (Block built = builder.build()) { - if (built instanceof AggregateMetricDoubleArrayBlock == false) { + if (built instanceof AggregateMetricDoubleArrayBlock == false && built instanceof DateRangeArrayBlock == false) { assertThat(built.asVector().isConstant(), is(true)); } assertThat(built, equalTo(random.block())); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java index 944b57f5eeeec..d9eeff27e532c 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorFixedBuilderTests.java @@ -34,7 +34,8 @@ public static List params() { || elementType == ElementType.NULL || elementType == ElementType.DOC || elementType == ElementType.BYTES_REF - || elementType == ElementType.AGGREGATE_METRIC_DOUBLE) { + || elementType == ElementType.AGGREGATE_METRIC_DOUBLE + || elementType == ElementType.DATE_RANGE) { continue; } params.add(new Object[] { elementType }); From 7c63d6df2200c648303cedfe8a772c5df31be3c8 Mon Sep 17 00:00:00 2001 From: Shmuel Hanoch Date: Fri, 10 Oct 2025 01:44:34 +0300 Subject: [PATCH 5/8] Two more tests fixes --- .../compute/data/BlockBuilderCopyFromTests.java | 2 +- .../compute/data/BlockValueAsserter.java | 9 +++++++++ .../compute/operator/topn/ExtractorTests.java | 11 +++++++++++ 3 files changed, 21 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java index 8b99ce973e886..dcce24de6c11d 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockBuilderCopyFromTests.java @@ -31,7 +31,7 @@ public static List params() { continue; } for (boolean nullAllowed : new boolean[] { false, true }) { - if (e == ElementType.AGGREGATE_METRIC_DOUBLE) { + if (e == ElementType.AGGREGATE_METRIC_DOUBLE || e == ElementType.DATE_RANGE) { // doesn't support multi-values params.add(new Object[] { e, nullAllowed, 0, 1 }); continue; diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockValueAsserter.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockValueAsserter.java index 82117eed95616..6b96b35d059d9 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockValueAsserter.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BlockValueAsserter.java @@ -42,6 +42,7 @@ static void assertBlockValues(Block block, List> expectedBlockValue valueCount, expectedRowValues ); + case DATE_RANGE -> assertDteRangeValues((DateRangeBlock) block, firstValueIndex, valueCount, expectedRowValues); default -> throw new IllegalArgumentException("Unsupported element type [" + block.elementType() + "]"); } } @@ -117,4 +118,12 @@ private static void assertAggregateMetricRowValues( assertThat(block.countBlock().getInt(firstValueIndex + valueIndex), is(equalTo(expectedValue.count()))); } } + + private static void assertDteRangeValues(DateRangeBlock block, int firstValueIdx, int valueCount, List expectedRowValues) { + for (int idx = 0; idx < valueCount; idx++) { + var expectedValue = (DateRangeBlockBuilder.DateRangeLiteral) expectedRowValues.get(idx); + assertThat(block.getFromBlock().getLong(firstValueIdx + idx), equalTo(expectedValue.from())); + assertThat(block.getToBlock().getLong(firstValueIdx + idx), equalTo(expectedValue.to())); + } + } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java index c5f91f05e76b3..9cc8dfd73f099 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.lucene.ShardRefCounted; @@ -31,6 +32,7 @@ import java.util.List; import java.util.function.Supplier; +import static org.elasticsearch.common.time.DateUtils.MAX_MILLIS_BEFORE_9999; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -64,6 +66,9 @@ public static Iterable parameters() { ) ); } + case DATE_RANGE -> cases.add( + valueTestCase("date_range with nulls", e, TopNEncoder.DEFAULT_UNSORTABLE, ExtractorTests::randomDateRange) + ); case FLOAT -> { } case BYTES_REF -> { @@ -256,4 +261,10 @@ public static AggregateMetricDoubleLiteral randomAggregateMetricDouble(boolean a randomBoolean() ? randomInt() : null ); } + + private static DateRangeBlockBuilder.DateRangeLiteral randomDateRange() { + var from = randomMillisUpToYear9999(); + var to = randomLongBetween(from + 1, MAX_MILLIS_BEFORE_9999); + return new DateRangeBlockBuilder.DateRangeLiteral(from, to); + } } From a813c7854fe0af68d402472e48425ad481f84547 Mon Sep 17 00:00:00 2001 From: Shmuel Hanoch Date: Fri, 10 Oct 2025 03:00:54 +0300 Subject: [PATCH 6/8] Two more tests --- .../compute/operator/topn/ResultBuilder.java | 1 + .../topn/ResultBuilderForDateRange.java | 56 +++++++++++++++++++ .../compute/data/VectorBuilderTests.java | 3 +- 3 files changed, 59 insertions(+), 1 deletion(-) create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java index c3da40254c09a..ffdfe618ac5d3 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java @@ -63,6 +63,7 @@ static ResultBuilder resultBuilderFor( case NULL -> new ResultBuilderForNull(blockFactory); case DOC -> new ResultBuilderForDoc(blockFactory, positions); case AGGREGATE_METRIC_DOUBLE -> new ResultBuilderForAggregateMetricDouble(blockFactory, positions); + case DATE_RANGE -> new ResultBuilderForDateRange(blockFactory, positions); default -> { assert false : "Result builder for [" + elementType + "]"; throw new UnsupportedOperationException("Result builder for [" + elementType + "]"); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java new file mode 100644 index 0000000000000..224cfc0a33910 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java @@ -0,0 +1,56 @@ +/* + * 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.compute.operator.topn; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.index.mapper.BlockLoader; + +import java.util.List; + +public class ResultBuilderForDateRange implements ResultBuilder { + + private final DateRangeBlockBuilder builder; + + ResultBuilderForDateRange(BlockFactory blockFactory, int positions) { + this.builder = blockFactory.newDateRangeBlockBuilder(positions); + } + + @Override + public void decodeKey(BytesRef keys) { + throw new AssertionError("DateRangeBlock can't be a key"); + } + + @Override + public void decodeValue(BytesRef values) { + int count = TopNEncoder.DEFAULT_UNSORTABLE.decodeVInt(values); + if (count == 0) { + builder.appendNull(); + } else { + builder.from().appendLong(TopNEncoder.DEFAULT_UNSORTABLE.decodeLong(values)); + builder.to().appendLong(TopNEncoder.DEFAULT_UNSORTABLE.decodeLong(values)); + } + } + + @Override + public Block build() { + return builder.build(); + } + + @Override + public String toString() { + return "ResultBuilderForDateRange"; + } + + @Override + public void close() { + builder.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java index 9a8c00d2c15de..2a1f72caafaad 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/VectorBuilderTests.java @@ -34,7 +34,8 @@ public static List params() { || e == ElementType.NULL || e == ElementType.DOC || e == ElementType.COMPOSITE - || e == ElementType.AGGREGATE_METRIC_DOUBLE) { + || e == ElementType.AGGREGATE_METRIC_DOUBLE + || e == ElementType.DATE_RANGE) { continue; } params.add(new Object[] { e }); From ee1bb7aa01aa76380d7eb2e1c161be937fede865 Mon Sep 17 00:00:00 2001 From: Shmuel Hanoch Date: Fri, 10 Oct 2025 15:05:15 +0300 Subject: [PATCH 7/8] Some more infra for new tests --- .../esql_date_range_created_version.csv | 2 +- .../resources/transport/upper_bounds/9.3.csv | 2 +- .../compute/operator/topn/ValueExtractor.java | 2 + .../topn/ValueExtractorForDateRange.java | 37 +++++++++++++++++++ .../compute/operator/topn/ExtractorTests.java | 11 +++--- .../operator/topn/TopNOperatorTests.java | 5 ++- .../compute/test/BlockTestUtils.java | 9 ++++- 7 files changed, 58 insertions(+), 10 deletions(-) create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDateRange.java diff --git a/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv b/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv index d617a6ff2f5d2..9a797d9f368cf 100644 --- a/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv +++ b/server/src/main/resources/transport/definitions/referable/esql_date_range_created_version.csv @@ -1 +1 @@ -9191000 +9192000 diff --git a/server/src/main/resources/transport/upper_bounds/9.3.csv b/server/src/main/resources/transport/upper_bounds/9.3.csv index 03e1a18b61d0a..a39e58f4029bb 100644 --- a/server/src/main/resources/transport/upper_bounds/9.3.csv +++ b/server/src/main/resources/transport/upper_bounds/9.3.csv @@ -1 +1 @@ -add_cross_cluster_api_key_signature,9191000 +esql_date_range_created_version,9192000 diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java index b6f3a1198d1ff..79546e9c0bfc6 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java @@ -11,6 +11,7 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BooleanBlock; import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.DateRangeBlock; import org.elasticsearch.compute.data.DocBlock; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.ElementType; @@ -53,6 +54,7 @@ static ValueExtractor extractorFor(ElementType elementType, TopNEncoder encoder, case NULL -> new ValueExtractorForNull(); case DOC -> new ValueExtractorForDoc(encoder, ((DocBlock) block).asVector()); case AGGREGATE_METRIC_DOUBLE -> new ValueExtractorForAggregateMetricDouble(encoder, (AggregateMetricDoubleBlock) block); + case DATE_RANGE -> new ValueExtractorForDateRange(encoder, (DateRangeBlock) block); default -> { assert false : "No value extractor for [" + block.elementType() + "]"; throw new UnsupportedOperationException("No value extractor for [" + block.elementType() + "]"); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDateRange.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDateRange.java new file mode 100644 index 0000000000000..b4ffb2a74056a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDateRange.java @@ -0,0 +1,37 @@ +/* + * 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.compute.operator.topn; + +import org.elasticsearch.compute.data.DateRangeBlock; +import org.elasticsearch.compute.operator.BreakingBytesRefBuilder; + +public class ValueExtractorForDateRange implements ValueExtractor { + private final DateRangeBlock block; + + ValueExtractorForDateRange(TopNEncoder encoder, DateRangeBlock block) { + assert encoder == TopNEncoder.DEFAULT_UNSORTABLE; + this.block = block; + } + + @Override + public void writeValue(BreakingBytesRefBuilder values, int position) { + TopNEncoder.DEFAULT_UNSORTABLE.encodeVInt(1, values); + if (block.isNull(position)) { + TopNEncoder.DEFAULT_UNSORTABLE.encodeBoolean(false, values); + } else { + TopNEncoder.DEFAULT_UNSORTABLE.encodeBoolean(true, values); + TopNEncoder.DEFAULT_UNSORTABLE.encodeDouble(block.getFromBlock().getLong(position), values); + TopNEncoder.DEFAULT_UNSORTABLE.encodeDouble(block.getToBlock().getLong(position), values); + } + } + + @Override + public String toString() { + return "ValueExtractorForDateRange"; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java index 9cc8dfd73f099..57b8b8ec7723d 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java @@ -66,9 +66,10 @@ public static Iterable parameters() { ) ); } - case DATE_RANGE -> cases.add( - valueTestCase("date_range with nulls", e, TopNEncoder.DEFAULT_UNSORTABLE, ExtractorTests::randomDateRange) - ); + case DATE_RANGE -> { + cases.add(valueTestCase("date_range with nulls", e, TopNEncoder.DEFAULT_UNSORTABLE, () -> randomDateRange(true))); + cases.add(valueTestCase("date_range with nulls", e, TopNEncoder.DEFAULT_UNSORTABLE, () -> randomDateRange(false))); + } case FLOAT -> { } case BYTES_REF -> { @@ -262,9 +263,9 @@ public static AggregateMetricDoubleLiteral randomAggregateMetricDouble(boolean a ); } - private static DateRangeBlockBuilder.DateRangeLiteral randomDateRange() { + private static DateRangeBlockBuilder.DateRangeLiteral randomDateRange(boolean isNull) { var from = randomMillisUpToYear9999(); var to = randomLongBetween(from + 1, MAX_MILLIS_BEFORE_9999); - return new DateRangeBlockBuilder.DateRangeLiteral(from, to); + return isNull ? null : new DateRangeBlockBuilder.DateRangeLiteral(from, to); } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java index 7d48b79ab12c5..f4c0e80bf93f2 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java @@ -74,6 +74,7 @@ import static org.elasticsearch.compute.data.ElementType.BOOLEAN; import static org.elasticsearch.compute.data.ElementType.BYTES_REF; import static org.elasticsearch.compute.data.ElementType.COMPOSITE; +import static org.elasticsearch.compute.data.ElementType.DATE_RANGE; import static org.elasticsearch.compute.data.ElementType.DOUBLE; import static org.elasticsearch.compute.data.ElementType.FLOAT; import static org.elasticsearch.compute.data.ElementType.INT; @@ -602,7 +603,7 @@ public void testCollectAllValues_RandomMultiValues() { for (int type = 0; type < blocksCount; type++) { ElementType e = randomFrom(ElementType.values()); - if (e == ElementType.UNKNOWN || e == COMPOSITE || e == AGGREGATE_METRIC_DOUBLE) { + if (e == ElementType.UNKNOWN || e == COMPOSITE || e == AGGREGATE_METRIC_DOUBLE || e == DATE_RANGE) { continue; } elementTypes.add(e); @@ -1024,7 +1025,7 @@ public void testRandomMultiValuesTopN() { for (int type = 0; type < blocksCount; type++) { ElementType e = randomValueOtherThanMany( - t -> t == ElementType.UNKNOWN || t == ElementType.DOC || t == COMPOSITE || t == AGGREGATE_METRIC_DOUBLE, + t -> t == ElementType.UNKNOWN || t == ElementType.DOC || t == COMPOSITE || t == AGGREGATE_METRIC_DOUBLE || t == DATE_RANGE, () -> randomFrom(ElementType.values()) ); elementTypes.add(e); diff --git a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java index 6f68b4f62a589..1b3b39b1fc0ca 100644 --- a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java +++ b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java @@ -35,6 +35,7 @@ import java.util.List; import java.util.Map; +import static org.elasticsearch.common.time.DateUtils.MAX_MILLIS_BEFORE_9999; import static org.elasticsearch.compute.data.BlockUtils.toJavaObject; import static org.elasticsearch.test.ESTestCase.between; import static org.elasticsearch.test.ESTestCase.randomBoolean; @@ -43,6 +44,8 @@ import static org.elasticsearch.test.ESTestCase.randomInt; import static org.elasticsearch.test.ESTestCase.randomIntBetween; import static org.elasticsearch.test.ESTestCase.randomLong; +import static org.elasticsearch.test.ESTestCase.randomLongBetween; +import static org.elasticsearch.test.ESTestCase.randomMillisUpToYear9999; import static org.elasticsearch.test.ESTestCase.randomNonNegativeInt; import static org.elasticsearch.test.ESTestCase.randomRealisticUnicodeOfCodepointLengthBetween; import static org.hamcrest.Matchers.equalTo; @@ -66,6 +69,11 @@ public static Object randomValue(ElementType e) { randomDouble(), randomNonNegativeInt() ); + case DATE_RANGE -> { + var from = randomMillisUpToYear9999(); + var to = randomLongBetween(from + 1, MAX_MILLIS_BEFORE_9999); + yield new DateRangeBlockBuilder.DateRangeLiteral(from, to); + } case DOC -> new BlockUtils.Doc( randomIntBetween(0, 255), // Shard ID should be small and non-negative. randomInt(), @@ -73,7 +81,6 @@ public static Object randomValue(ElementType e) { ); case NULL -> null; case COMPOSITE -> throw new IllegalArgumentException("can't make random values for composite"); - case DATE_RANGE -> throw new IllegalArgumentException("can't make random values for date range"); case UNKNOWN -> throw new IllegalArgumentException("can't make random values for [" + e + "]"); }; } From e4eb387282417c73f7a5bfc8743f4ea878dd4d0b Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Fri, 10 Oct 2025 12:21:22 +0000 Subject: [PATCH 8/8] [CI] Auto commit changes from spotless --- .../compute/operator/topn/ResultBuilderForDateRange.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java index 224cfc0a33910..23f9da1fe314f 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDateRange.java @@ -8,12 +8,9 @@ package org.elasticsearch.compute.operator.topn; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.compute.data.DateRangeBlockBuilder; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; -import org.elasticsearch.index.mapper.BlockLoader; - -import java.util.List; +import org.elasticsearch.compute.data.DateRangeBlockBuilder; public class ResultBuilderForDateRange implements ResultBuilder {