diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java index 9c527923fae02..1827babe6f091 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java @@ -24,28 +24,19 @@ import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.DoubleVector; -import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.LongVector; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.BlockReaderFactories; import org.elasticsearch.compute.lucene.LuceneSourceOperator; -import org.elasticsearch.compute.lucene.ValueSourceInfo; import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; import org.elasticsearch.compute.operator.topn.TopNOperator; import org.elasticsearch.core.IOUtils; -import org.elasticsearch.index.fielddata.FieldData; -import org.elasticsearch.index.fielddata.IndexFieldDataCache; -import org.elasticsearch.index.fielddata.IndexNumericFieldData; -import org.elasticsearch.index.fielddata.plain.SortedDoublesIndexFieldData; -import org.elasticsearch.index.fielddata.plain.SortedNumericIndexFieldData; -import org.elasticsearch.index.fielddata.plain.SortedSetOrdinalsIndexFieldData; +import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.KeywordFieldMapper; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.script.field.KeywordDocValuesField; -import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; -import org.elasticsearch.search.aggregations.support.FieldContext; +import org.elasticsearch.index.mapper.NumberFieldMapper; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -101,54 +92,18 @@ public class ValuesSourceReaderBenchmark { } } - private static ValueSourceInfo info(IndexReader reader, String name) { + private static BlockLoader blockLoader(String name) { return switch (name) { - case "long" -> numericInfo(reader, name, IndexNumericFieldData.NumericType.LONG, ElementType.LONG); - case "int" -> numericInfo(reader, name, IndexNumericFieldData.NumericType.INT, ElementType.INT); - case "double" -> { - SortedDoublesIndexFieldData fd = new SortedDoublesIndexFieldData( - name, - IndexNumericFieldData.NumericType.DOUBLE, - CoreValuesSourceType.NUMERIC, - null - ); - FieldContext context = new FieldContext(name, fd, null); - yield new ValueSourceInfo( - CoreValuesSourceType.NUMERIC, - CoreValuesSourceType.NUMERIC.getField(context, null), - ElementType.DOUBLE, - reader - ); - } - case "keyword" -> { - SortedSetOrdinalsIndexFieldData fd = new SortedSetOrdinalsIndexFieldData( - new IndexFieldDataCache.None(), - "keyword", - CoreValuesSourceType.KEYWORD, - new NoneCircuitBreakerService(), - (dv, n) -> new KeywordDocValuesField(FieldData.toString(dv), n) - ); - FieldContext context = new FieldContext(name, fd, null); - yield new ValueSourceInfo( - CoreValuesSourceType.KEYWORD, - CoreValuesSourceType.KEYWORD.getField(context, null), - ElementType.BYTES_REF, - reader - ); - } + case "long" -> numericBlockLoader(name, NumberFieldMapper.NumberType.LONG); + case "int" -> numericBlockLoader(name, NumberFieldMapper.NumberType.INTEGER); + case "double" -> numericBlockLoader(name, NumberFieldMapper.NumberType.DOUBLE); + case "keyword" -> new KeywordFieldMapper.KeywordFieldType(name).blockLoader(null); default -> throw new IllegalArgumentException("can't read [" + name + "]"); }; } - private static ValueSourceInfo numericInfo( - IndexReader reader, - String name, - IndexNumericFieldData.NumericType numericType, - ElementType elementType - ) { - SortedNumericIndexFieldData fd = new SortedNumericIndexFieldData(name, numericType, CoreValuesSourceType.NUMERIC, null); - FieldContext context = new FieldContext(name, fd, null); - return new ValueSourceInfo(CoreValuesSourceType.NUMERIC, CoreValuesSourceType.NUMERIC.getField(context, null), elementType, reader); + private static BlockLoader numericBlockLoader(String name, NumberFieldMapper.NumberType numberType) { + return new NumberFieldMapper.NumberFieldType(name, numberType).blockLoader(null); } /** @@ -176,7 +131,11 @@ private static ValueSourceInfo numericInfo( @Benchmark @OperationsPerInvocation(INDEX_SIZE) public void benchmark() { - ValuesSourceReaderOperator op = new ValuesSourceReaderOperator(List.of(info(reader, name)), 0, name); + ValuesSourceReaderOperator op = new ValuesSourceReaderOperator( + List.of(BlockReaderFactories.loaderToFactory(reader, blockLoader(name))), + 0, + name + ); long sum = 0; for (Page page : pages) { op.addInput(page); @@ -203,13 +162,24 @@ public void benchmark() { BytesRef scratch = new BytesRef(); BytesRefVector values = op.getOutput().getBlock(1).asVector(); for (int p = 0; p < values.getPositionCount(); p++) { - sum += Integer.parseInt(values.getBytesRef(p, scratch).utf8ToString()); + BytesRef r = values.getBytesRef(p, scratch); + r.offset++; + r.length--; + sum += Integer.parseInt(r.utf8ToString()); } } } } - long expected = INDEX_SIZE; - expected = expected * (expected - 1) / 2; + long expected; + if (name.equals("keyword")) { + expected = 0; + for (int i = 0; i < INDEX_SIZE; i++) { + expected += i % 1000; + } + } else { + expected = INDEX_SIZE; + expected = expected * (expected - 1) / 2; + } if (expected != sum) { throw new AssertionError("[" + layout + "][" + name + "] expected [" + expected + "] but was [" + sum + "]"); } @@ -225,16 +195,13 @@ private void setupIndex() throws IOException { directory = new ByteBuffersDirectory(); try (IndexWriter iw = new IndexWriter(directory, new IndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE))) { for (int i = 0; i < INDEX_SIZE; i++) { + String c = Character.toString('a' - ((i % 1000) % 26) + 26); iw.addDocument( List.of( new NumericDocValuesField("long", i), new NumericDocValuesField("int", i), new NumericDocValuesField("double", NumericUtils.doubleToSortableLong(i)), - new KeywordFieldMapper.KeywordField( - "keyword", - new BytesRef(Integer.toString(i)), - KeywordFieldMapper.Defaults.FIELD_TYPE - ) + new KeywordFieldMapper.KeywordField("keyword", new BytesRef(c + i % 1000), KeywordFieldMapper.Defaults.FIELD_TYPE) ) ); if (i % COMMIT_INTERVAL == 0) { diff --git a/docs/changelog/101235.yaml b/docs/changelog/101235.yaml new file mode 100644 index 0000000000000..53adf9527c2c4 --- /dev/null +++ b/docs/changelog/101235.yaml @@ -0,0 +1,5 @@ +pr: 101235 +summary: Load different way +area: ES|QL +type: enhancement +issues: [] diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapper.java b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapper.java index fa7c61cdd80b6..ee04346591009 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapper.java +++ b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapper.java @@ -39,6 +39,9 @@ import org.elasticsearch.index.fielddata.StoredFieldSortedBinaryIndexFieldData; import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; +import org.elasticsearch.index.mapper.BlockLoader; +import org.elasticsearch.index.mapper.BlockSourceReader; +import org.elasticsearch.index.mapper.BlockStoredFieldsReader; import org.elasticsearch.index.mapper.DocumentParserContext; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperBuilderContext; @@ -318,6 +321,14 @@ public Query phrasePrefixQuery(TokenStream stream, int slop, int maxExpansions, return toQuery(query, queryShardContext); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (textFieldType.isSyntheticSource()) { + return BlockStoredFieldsReader.bytesRefsFromStrings(storedFieldNameForSyntheticSource()); + } + return BlockSourceReader.bytesRefs(SourceValueFetcher.toString(blContext.sourcePaths(name()))); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { if (fieldDataContext.fielddataOperation() != FielddataOperation.SCRIPT) { diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapper.java b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapper.java index 43951878933fa..abed23621d5e9 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapper.java +++ b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapper.java @@ -30,6 +30,9 @@ import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SourceValueFetcherSortedDoubleIndexFieldData; import org.elasticsearch.index.fielddata.plain.SortedNumericIndexFieldData; +import org.elasticsearch.index.mapper.BlockDocValuesReader; +import org.elasticsearch.index.mapper.BlockLoader; +import org.elasticsearch.index.mapper.BlockSourceReader; import org.elasticsearch.index.mapper.DocumentParserContext; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperBuilderContext; @@ -303,6 +306,19 @@ public Query rangeQuery( return NumberFieldMapper.NumberType.LONG.rangeQuery(name(), lo, hi, true, true, hasDocValues(), context, isIndexed()); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (indexMode == IndexMode.TIME_SERIES && metricType == TimeSeriesParams.MetricType.COUNTER) { + // Counters are not supported by ESQL so we load them in null + return BlockDocValuesReader.nulls(); + } + if (hasDocValues()) { + double scalingFactorInverse = 1d / scalingFactor; + return BlockDocValuesReader.doubles(name(), l -> l * scalingFactorInverse); + } + return BlockSourceReader.doubles(sourceValueFetcher(blContext.sourcePaths(name()))); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { FielddataOperation operation = fieldDataContext.fielddataOperation(); diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapperTests.java index 4c738ddcffce1..e1fbc2e149441 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/MatchOnlyTextFieldMapperTests.java @@ -20,6 +20,7 @@ import org.apache.lucene.tests.analysis.CannedTokenStream; import org.apache.lucene.tests.analysis.Token; import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.Strings; import org.elasticsearch.core.Tuple; import org.elasticsearch.index.mapper.DocumentMapper; @@ -42,6 +43,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.function.Function; import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; @@ -261,4 +263,9 @@ public void testDocValuesLoadedFromSynthetic() throws IOException { protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); } + + @Override + protected Function loadBlockExpected() { + return v -> ((BytesRef) v).utf8ToString(); + } } diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java index ad0610732b758..665e9289c3c7d 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java @@ -30,17 +30,20 @@ import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentFactory; import org.elasticsearch.xcontent.XContentType; +import org.hamcrest.Matcher; import org.junit.AssumptionViolatedException; import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.function.Function; import static java.util.Collections.singletonList; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notANumber; public class ScaledFloatFieldMapperTests extends MapperTestCase { @@ -368,13 +371,15 @@ private static class ScaledFloatSyntheticSourceSupport implements SyntheticSourc public SyntheticSourceExample example(int maxValues) { if (randomBoolean()) { Tuple v = generateValue(); - return new SyntheticSourceExample(v.v1(), v.v2(), this::mapping); + return new SyntheticSourceExample(v.v1(), v.v2(), roundDocValues(v.v2()), this::mapping); } List> values = randomList(1, maxValues, this::generateValue); List in = values.stream().map(Tuple::v1).toList(); List outList = values.stream().map(Tuple::v2).sorted().toList(); Object out = outList.size() == 1 ? outList.get(0) : outList; - return new SyntheticSourceExample(in, out, this::mapping); + List outBlockList = values.stream().map(v -> roundDocValues(v.v2())).sorted().toList(); + Object outBlock = outBlockList.size() == 1 ? outBlockList.get(0) : outBlockList; + return new SyntheticSourceExample(in, out, outBlock, this::mapping); } private Tuple generateValue() { @@ -398,6 +403,11 @@ private double round(double d) { return decoded; } + private double roundDocValues(double d) { + long encoded = Math.round(d * scalingFactor); + return encoded * (1 / scalingFactor); + } + private void mapping(XContentBuilder b) throws IOException { b.field("type", "scaled_float"); b.field("scaling_factor", scalingFactor); @@ -427,6 +437,16 @@ public List invalidExample() throws IOException { } } + @Override + protected Function loadBlockExpected() { + return v -> (Number) v; + } + + @Override + protected Matcher blockItemMatcher(Object expected) { + return "NaN".equals(expected) ? notANumber() : equalTo(expected); + } + @Override protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataService.java b/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataService.java index e4030a0de9d61..27ef30ed67508 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataService.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/IndexFieldDataService.java @@ -91,8 +91,12 @@ public synchronized void clearField(final String fieldName) { */ @SuppressWarnings("unchecked") public > IFD getForField(MappedFieldType fieldType, FieldDataContext fieldDataContext) { + return getFromBuilder(fieldType, fieldType.fielddataBuilder(fieldDataContext)); + } + + @SuppressWarnings("unchecked") + public > IFD getFromBuilder(MappedFieldType fieldType, IndexFieldData.Builder builder) { final String fieldName = fieldType.name(); - IndexFieldData.Builder builder = fieldType.fielddataBuilder(fieldDataContext); IndexFieldDataCache cache; synchronized (this) { cache = fieldDataCaches.get(fieldName); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BlockDocValuesReader.java b/server/src/main/java/org/elasticsearch/index/mapper/BlockDocValuesReader.java new file mode 100644 index 0000000000000..049a779c97503 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/BlockDocValuesReader.java @@ -0,0 +1,857 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.NumericDocValues; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.UnicodeUtil; +import org.elasticsearch.core.CheckedFunction; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.mapper.BlockLoader.BooleanBuilder; +import org.elasticsearch.index.mapper.BlockLoader.Builder; +import org.elasticsearch.index.mapper.BlockLoader.BuilderFactory; +import org.elasticsearch.index.mapper.BlockLoader.BytesRefBuilder; +import org.elasticsearch.index.mapper.BlockLoader.Docs; +import org.elasticsearch.index.mapper.BlockLoader.DoubleBuilder; +import org.elasticsearch.index.mapper.BlockLoader.IntBuilder; +import org.elasticsearch.index.mapper.BlockLoader.LongBuilder; +import org.elasticsearch.index.mapper.BlockLoader.SingletonOrdinalsBuilder; + +import java.io.IOException; + +/** + * A reader that supports reading doc-values from a Lucene segment in Block fashion. + */ +public abstract class BlockDocValuesReader { + public interface Factory { + BlockDocValuesReader build(int segment) throws IOException; + + boolean supportsOrdinals(); + + SortedSetDocValues ordinals(int segment) throws IOException; + } + + protected final Thread creationThread; + + public BlockDocValuesReader() { + this.creationThread = Thread.currentThread(); + } + + /** + * Returns the current doc that this reader is on. + */ + public abstract int docID(); + + /** + * The {@link BlockLoader.Builder} for data of this type. + */ + public abstract Builder builder(BuilderFactory factory, int expectedCount); + + /** + * Reads the values of the given documents specified in the input block + */ + public abstract Builder readValues(BuilderFactory factory, Docs docs) throws IOException; + + /** + * Reads the values of the given document into the builder + */ + public abstract void readValuesFromSingleDoc(int docId, Builder builder) throws IOException; + + /** + * Checks if the reader can be used to read a range documents starting with the given docID by the current thread. + */ + public static boolean canReuse(BlockDocValuesReader reader, int startingDocID) { + return reader != null && reader.creationThread == Thread.currentThread() && reader.docID() <= startingDocID; + } + + public static BlockLoader booleans(String fieldName) { + return context -> { + SortedNumericDocValues docValues = DocValues.getSortedNumeric(context.reader(), fieldName); + NumericDocValues singleton = DocValues.unwrapSingleton(docValues); + if (singleton != null) { + return new SingletonBooleans(singleton); + } + return new Booleans(docValues); + }; + } + + public static BlockLoader bytesRefsFromOrds(String fieldName) { + return new BlockLoader() { + @Override + public BlockDocValuesReader reader(LeafReaderContext context) throws IOException { + SortedSetDocValues docValues = ordinals(context); + SortedDocValues singleton = DocValues.unwrapSingleton(docValues); + if (singleton != null) { + return new SingletonOrdinals(singleton); + } + return new Ordinals(docValues); + } + + @Override + public boolean supportsOrdinals() { + return true; + } + + @Override + public SortedSetDocValues ordinals(LeafReaderContext context) throws IOException { + return DocValues.getSortedSet(context.reader(), fieldName); + } + }; + } + + /** + * Load {@link BytesRef} values from doc values. Prefer {@link #bytesRefsFromOrds} if + * doc values are indexed with ordinals because that's generally much faster. It's + * possible to use this with field data, but generally should be avoided because field + * data has higher per invocation overhead. + */ + public static BlockLoader bytesRefsFromDocValues(CheckedFunction fieldData) { + return context -> new Bytes(fieldData.apply(context)); + } + + /** + * Convert from the stored {@link long} into the {@link double} to load. + * Sadly, this will go megamorphic pretty quickly and slow us down, + * but it gets the job done for now. + */ + public interface ToDouble { + double convert(long v); + } + + /** + * Load {@code double} values from doc values. + */ + public static BlockLoader doubles(String fieldName, ToDouble toDouble) { + return context -> { + SortedNumericDocValues docValues = DocValues.getSortedNumeric(context.reader(), fieldName); + NumericDocValues singleton = DocValues.unwrapSingleton(docValues); + if (singleton != null) { + return new SingletonDoubles(singleton, toDouble); + } + return new Doubles(docValues, toDouble); + }; + } + + /** + * Load {@code int} values from doc values. + */ + public static BlockLoader ints(String fieldName) { + return context -> { + SortedNumericDocValues docValues = DocValues.getSortedNumeric(context.reader(), fieldName); + NumericDocValues singleton = DocValues.unwrapSingleton(docValues); + if (singleton != null) { + return new SingletonInts(singleton); + } + return new Ints(docValues); + }; + } + + /** + * Load a block of {@code long}s from doc values. + */ + public static BlockLoader longs(String fieldName) { + return context -> { + SortedNumericDocValues docValues = DocValues.getSortedNumeric(context.reader(), fieldName); + NumericDocValues singleton = DocValues.unwrapSingleton(docValues); + if (singleton != null) { + return new SingletonLongs(singleton); + } + return new Longs(docValues); + }; + } + + /** + * Load blocks with only null. + */ + public static BlockLoader nulls() { + return context -> new Nulls(); + } + + @Override + public abstract String toString(); + + private static class SingletonLongs extends BlockDocValuesReader { + private final NumericDocValues numericDocValues; + + SingletonLongs(NumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public LongBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.longsFromDocValues(expectedCount); + } + + @Override + public LongBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + int lastDoc = -1; + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (numericDocValues.advanceExact(doc)) { + blockBuilder.appendLong(numericDocValues.longValue()); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + BlockLoader.LongBuilder blockBuilder = (BlockLoader.LongBuilder) builder; + if (numericDocValues.advanceExact(docId)) { + blockBuilder.appendLong(numericDocValues.longValue()); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return numericDocValues.docID(); + } + + @Override + public String toString() { + return "SingletonLongs"; + } + } + + private static class Longs extends BlockDocValuesReader { + private final SortedNumericDocValues numericDocValues; + private int docID = -1; + + Longs(SortedNumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public BlockLoader.LongBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.longsFromDocValues(expectedCount); + } + + @Override + public BlockLoader.LongBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + read(docId, (LongBuilder) builder); + } + + private void read(int doc, LongBuilder builder) throws IOException { + this.docID = doc; + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = numericDocValues.docValueCount(); + if (count == 1) { + builder.appendLong(numericDocValues.nextValue()); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendLong(numericDocValues.nextValue()); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + // There is a .docID on the numericDocValues but it is often not implemented. + return docID; + } + + @Override + public String toString() { + return "Longs"; + } + } + + private static class SingletonInts extends BlockDocValuesReader { + private final NumericDocValues numericDocValues; + + SingletonInts(NumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public IntBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.intsFromDocValues(expectedCount); + } + + @Override + public IntBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + int lastDoc = -1; + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (numericDocValues.advanceExact(doc)) { + blockBuilder.appendInt(Math.toIntExact(numericDocValues.longValue())); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + IntBuilder blockBuilder = (IntBuilder) builder; + if (numericDocValues.advanceExact(docId)) { + blockBuilder.appendInt(Math.toIntExact(numericDocValues.longValue())); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return numericDocValues.docID(); + } + + @Override + public String toString() { + return "SingletonInts"; + } + } + + private static class Ints extends BlockDocValuesReader { + private final SortedNumericDocValues numericDocValues; + private int docID = -1; + + Ints(SortedNumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public IntBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.intsFromDocValues(expectedCount); + } + + @Override + public IntBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + read(docId, (IntBuilder) builder); + } + + private void read(int doc, IntBuilder builder) throws IOException { + this.docID = doc; + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = numericDocValues.docValueCount(); + if (count == 1) { + builder.appendInt(Math.toIntExact(numericDocValues.nextValue())); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendInt(Math.toIntExact(numericDocValues.nextValue())); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + // There is a .docID on on the numericDocValues but it is often not implemented. + return docID; + } + + @Override + public String toString() { + return "Ints"; + } + } + + private static class SingletonDoubles extends BlockDocValuesReader { + private final NumericDocValues docValues; + private final ToDouble toDouble; + private int docID = -1; + + SingletonDoubles(NumericDocValues docValues, ToDouble toDouble) { + this.docValues = docValues; + this.toDouble = toDouble; + } + + @Override + public DoubleBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.doublesFromDocValues(expectedCount); + } + + @Override + public DoubleBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + int lastDoc = -1; + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (docValues.advanceExact(doc)) { + blockBuilder.appendDouble(toDouble.convert(docValues.longValue())); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + this.docID = doc; + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + this.docID = docId; + DoubleBuilder blockBuilder = (DoubleBuilder) builder; + if (docValues.advanceExact(this.docID)) { + blockBuilder.appendDouble(toDouble.convert(docValues.longValue())); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return "SingletonDoubles"; + } + } + + private static class Doubles extends BlockDocValuesReader { + private final SortedNumericDocValues docValues; + private final ToDouble toDouble; + private int docID = -1; + + Doubles(SortedNumericDocValues docValues, ToDouble toDouble) { + this.docValues = docValues; + this.toDouble = toDouble; + } + + @Override + public DoubleBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.doublesFromDocValues(expectedCount); + } + + @Override + public DoubleBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + read(docId, (DoubleBuilder) builder); + } + + private void read(int doc, DoubleBuilder builder) throws IOException { + this.docID = doc; + if (false == docValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = docValues.docValueCount(); + if (count == 1) { + builder.appendDouble(toDouble.convert(docValues.nextValue())); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendDouble(toDouble.convert(docValues.nextValue())); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return "Doubles"; + } + } + + private static class SingletonOrdinals extends BlockDocValuesReader { + private final SortedDocValues ordinals; + + SingletonOrdinals(SortedDocValues ordinals) { + this.ordinals = ordinals; + } + + @Override + public BytesRefBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.bytesRefsFromDocValues(expectedCount); + } + + @Override + public SingletonOrdinalsBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + SingletonOrdinalsBuilder builder = factory.singletonOrdinalsBuilder(ordinals, docs.count()); + + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < ordinals.docID()) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (ordinals.advanceExact(doc)) { + builder.appendOrd(ordinals.ordValue()); + } else { + builder.appendNull(); + } + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int doc, Builder builder) throws IOException { + if (ordinals.advanceExact(doc)) { + ((BytesRefBuilder) builder).appendBytesRef(ordinals.lookupOrd(ordinals.ordValue())); + } else { + builder.appendNull(); + } + } + + @Override + public int docID() { + return ordinals.docID(); + } + + @Override + public String toString() { + return "SingletonOrdinals"; + } + } + + private static class Ordinals extends BlockDocValuesReader { + private final SortedSetDocValues ordinals; + + Ordinals(SortedSetDocValues ordinals) { + this.ordinals = ordinals; + } + + @Override + public BytesRefBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.bytesRefsFromDocValues(expectedCount); + } + + @Override + public BytesRefBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + BytesRefBuilder builder = builder(factory, docs.count()); + + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < ordinals.docID()) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, builder); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int doc, Builder builder) throws IOException { + read(doc, (BytesRefBuilder) builder); + } + + private void read(int doc, BytesRefBuilder builder) throws IOException { + if (false == ordinals.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = ordinals.docValueCount(); + if (count == 1) { + builder.appendBytesRef(ordinals.lookupOrd(ordinals.nextOrd())); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendBytesRef(ordinals.lookupOrd(ordinals.nextOrd())); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + return ordinals.docID(); + } + + @Override + public String toString() { + return "Ordinals"; + } + } + + private static class Bytes extends BlockDocValuesReader { + private final SortedBinaryDocValues docValues; + private int docID = -1; + + Bytes(SortedBinaryDocValues docValues) { + this.docValues = docValues; + } + + @Override + public BytesRefBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.bytesRefsFromDocValues(expectedCount); + } + + @Override + public BytesRefBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + read(docId, (BytesRefBuilder) builder); + } + + private void read(int doc, BytesRefBuilder builder) throws IOException { + this.docID = doc; + if (false == docValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = docValues.docValueCount(); + if (count == 1) { + // TODO read ords in ascending order. Buffers and stuff. + builder.appendBytesRef(docValues.nextValue()); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendBytesRef(docValues.nextValue()); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return "Bytes"; + } + } + + private static class SingletonBooleans extends BlockDocValuesReader { + private final NumericDocValues numericDocValues; + + SingletonBooleans(NumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public BooleanBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.booleansFromDocValues(expectedCount); + } + + @Override + public BooleanBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + int lastDoc = -1; + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < lastDoc) { + throw new IllegalStateException("docs within same block must be in order"); + } + if (numericDocValues.advanceExact(doc)) { + blockBuilder.appendBoolean(numericDocValues.longValue() != 0); + } else { + blockBuilder.appendNull(); + } + lastDoc = doc; + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + BooleanBuilder blockBuilder = (BooleanBuilder) builder; + if (numericDocValues.advanceExact(docId)) { + blockBuilder.appendBoolean(numericDocValues.longValue() != 0); + } else { + blockBuilder.appendNull(); + } + } + + @Override + public int docID() { + return numericDocValues.docID(); + } + + @Override + public String toString() { + return "SingletonBooleans"; + } + } + + private static class Booleans extends BlockDocValuesReader { + private final SortedNumericDocValues numericDocValues; + private int docID = -1; + + Booleans(SortedNumericDocValues numericDocValues) { + this.numericDocValues = numericDocValues; + } + + @Override + public BooleanBuilder builder(BuilderFactory factory, int expectedCount) { + return factory.booleansFromDocValues(expectedCount); + } + + @Override + public BooleanBuilder readValues(BuilderFactory factory, Docs docs) throws IOException { + var blockBuilder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + read(doc, blockBuilder); + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) throws IOException { + read(docId, (BooleanBuilder) builder); + } + + private void read(int doc, BooleanBuilder builder) throws IOException { + this.docID = doc; + if (false == numericDocValues.advanceExact(doc)) { + builder.appendNull(); + return; + } + int count = numericDocValues.docValueCount(); + if (count == 1) { + builder.appendBoolean(numericDocValues.nextValue() != 0); + return; + } + builder.beginPositionEntry(); + for (int v = 0; v < count; v++) { + builder.appendBoolean(numericDocValues.nextValue() != 0); + } + builder.endPositionEntry(); + } + + @Override + public int docID() { + // There is a .docID on the numericDocValues but it is often not implemented. + return docID; + } + + @Override + public String toString() { + return "Booleans"; + } + } + + private static class Nulls extends BlockDocValuesReader { + private int docID = -1; + + @Override + public Builder builder(BuilderFactory factory, int expectedCount) { + return factory.nulls(expectedCount); + } + + @Override + public Builder readValues(BuilderFactory factory, Docs docs) throws IOException { + Builder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + builder.appendNull(); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, Builder builder) { + this.docID = docId; + builder.appendNull(); + } + + @Override + public int docID() { + return docID; + } + + @Override + public String toString() { + return "Nulls"; + } + } + + /** + * Convert a {@link String} into a utf-8 {@link BytesRef}. + */ + protected static BytesRef toBytesRef(BytesRef scratch, String v) { + int len = UnicodeUtil.maxUTF8Length(v.length()); + if (scratch.bytes.length < len) { + scratch.bytes = new byte[len]; + } + scratch.length = UnicodeUtil.UTF16toUTF8(v, 0, v.length(), scratch.bytes); + return scratch; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BlockLoader.java b/server/src/main/java/org/elasticsearch/index/mapper/BlockLoader.java new file mode 100644 index 0000000000000..7e973f9c32033 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/BlockLoader.java @@ -0,0 +1,191 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.util.BytesRef; + +import java.io.IOException; + +/** + * Interface for loading data in a block shape. Instances of this class + * must be immutable and thread safe. + */ +public interface BlockLoader { + /** + * Build a {@link LeafReaderContext leaf} level reader. + */ + BlockDocValuesReader reader(LeafReaderContext context) throws IOException; + + /** + * Does this loader support loading bytes via calling {@link #ordinals}. + */ + default boolean supportsOrdinals() { + return false; + } + + /** + * Load ordinals for the provided context. + */ + default SortedSetDocValues ordinals(LeafReaderContext context) throws IOException { + throw new IllegalStateException("ordinals not supported"); + } + + /** + * A list of documents to load. + */ + interface Docs { + int count(); + + int get(int i); + } + + /** + * Builds block "builders" for loading data into blocks for the compute engine. + * It's important for performance that this only have one implementation in + * production code. That implementation sits in the "compute" project. The is + * also a test implementation, but there may be no more other implementations. + */ + interface BuilderFactory { + /** + * Build a builder to load booleans as loaded from doc values. Doc values + * load booleans deduplicated and in sorted order. + */ + BooleanBuilder booleansFromDocValues(int expectedCount); + + /** + * Build a builder to load booleans without any loading constraints. + */ + BooleanBuilder booleans(int expectedCount); + + /** + * Build a builder to load {@link BytesRef}s as loaded from doc values. + * Doc values load {@linkplain BytesRef}s deduplicated and in sorted order. + */ + BytesRefBuilder bytesRefsFromDocValues(int expectedCount); + + /** + * Build a builder to load {@link BytesRef}s without any loading constraints. + */ + BytesRefBuilder bytesRefs(int expectedCount); + + /** + * Build a builder to load doubles as loaded from doc values. + * Doc values load doubles deduplicated and in sorted order. + */ + DoubleBuilder doublesFromDocValues(int expectedCount); + + /** + * Build a builder to load doubles without any loading constraints. + */ + DoubleBuilder doubles(int expectedCount); + + /** + * Build a builder to load ints as loaded from doc values. + * Doc values load ints deduplicated and in sorted order. + */ + IntBuilder intsFromDocValues(int expectedCount); + + /** + * Build a builder to load ints without any loading constraints. + */ + IntBuilder ints(int expectedCount); + + /** + * Build a builder to load longs as loaded from doc values. + * Doc values load longs deduplicated and in sorted order. + */ + LongBuilder longsFromDocValues(int expectedCount); + + /** + * Build a builder to load longs without any loading constraints. + */ + LongBuilder longs(int expectedCount); + + /** + * Build a builder that can only load null values. + * TODO this should return a block directly instead of a builder + */ + Builder nulls(int expectedCount); + + /** + * Build a reader for reading keyword ordinals. + */ + SingletonOrdinalsBuilder singletonOrdinalsBuilder(SortedDocValues ordinals, int count); + + // TODO support non-singleton ords + } + + /** + * A builder for typed values. For each document you may either call + * {@link #appendNull}, {@code append}, or + * {@link #beginPositionEntry} followed by two or more {@code append} + * calls, and then {@link #endPositionEntry}. + */ + interface Builder { + /** + * Insert a null value. + */ + Builder appendNull(); + + /** + * Start a multivalued field. + */ + Builder beginPositionEntry(); + + /** + * End a multivalued field. + */ + Builder endPositionEntry(); + } + + interface BooleanBuilder extends Builder { + /** + * Appends a boolean to the current entry. + */ + BooleanBuilder appendBoolean(boolean value); + } + + interface BytesRefBuilder extends Builder { + /** + * Appends a BytesRef to the current entry. + */ + BytesRefBuilder appendBytesRef(BytesRef value); + } + + interface DoubleBuilder extends Builder { + /** + * Appends a double to the current entry. + */ + DoubleBuilder appendDouble(double value); + } + + interface IntBuilder extends Builder { + /** + * Appends an int to the current entry. + */ + IntBuilder appendInt(int value); + } + + interface LongBuilder extends Builder { + /** + * Appends a long to the current entry. + */ + LongBuilder appendLong(long value); + } + + interface SingletonOrdinalsBuilder extends Builder { + /** + * Appends an ordinal to the builder. + */ + SingletonOrdinalsBuilder appendOrd(int value); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BlockSourceReader.java b/server/src/main/java/org/elasticsearch/index/mapper/BlockSourceReader.java new file mode 100644 index 0000000000000..2b9daadda31d6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/BlockSourceReader.java @@ -0,0 +1,198 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; +import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; +import org.elasticsearch.search.lookup.Source; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +/** + * Loads values from {@code _source}. This whole process is very slow and cast-tastic, + * so it doesn't really try to avoid megamorphic invocations. It's just going to be + * slow. + * + * Note that this extends {@link BlockDocValuesReader} because it pretends to load + * doc values because, for now, ESQL only knows how to load things in a doc values + * order. + */ +public abstract class BlockSourceReader extends BlockDocValuesReader { + /** + * Read {@code boolean}s from {@code _source}. + */ + public static BlockLoader booleans(ValueFetcher fetcher) { + StoredFieldLoader loader = StoredFieldLoader.create(true, Set.of()); + return context -> new BlockSourceReader(fetcher, loader.getLoader(context, null)) { + @Override + public BlockLoader.Builder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.booleans(expectedCount); + } + + @Override + protected void append(BlockLoader.Builder builder, Object v) { + ((BlockLoader.BooleanBuilder) builder).appendBoolean((Boolean) v); + } + + @Override + public String toString() { + return "SourceBooleans"; + } + }; + } + + /** + * Read {@link BytesRef}s from {@code _source}. + */ + public static BlockLoader bytesRefs(ValueFetcher fetcher) { + StoredFieldLoader loader = StoredFieldLoader.create(true, Set.of()); + return context -> new BlockSourceReader(fetcher, loader.getLoader(context, null)) { + BytesRef scratch = new BytesRef(); + + @Override + public BlockLoader.Builder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.bytesRefs(expectedCount); + } + + @Override + protected void append(BlockLoader.Builder builder, Object v) { + ((BlockLoader.BytesRefBuilder) builder).appendBytesRef(toBytesRef(scratch, (String) v)); + } + + @Override + public String toString() { + return "SourceBytes"; + } + }; + } + + /** + * Read {@code double}s from {@code _source}. + */ + public static BlockLoader doubles(ValueFetcher fetcher) { + StoredFieldLoader loader = StoredFieldLoader.create(true, Set.of()); + return context -> new BlockSourceReader(fetcher, loader.getLoader(context, null)) { + @Override + public BlockLoader.Builder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.doubles(expectedCount); + } + + @Override + protected void append(BlockLoader.Builder builder, Object v) { + ((BlockLoader.DoubleBuilder) builder).appendDouble(((Number) v).doubleValue()); + } + + @Override + public String toString() { + return "SourceDoubles"; + } + }; + } + + /** + * Read {@code int}s from {@code _source}. + */ + public static BlockLoader ints(ValueFetcher fetcher) { + StoredFieldLoader loader = StoredFieldLoader.create(true, Set.of()); + return context -> new BlockSourceReader(fetcher, loader.getLoader(context, null)) { + @Override + public BlockLoader.Builder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.ints(expectedCount); + } + + @Override + protected void append(BlockLoader.Builder builder, Object v) { + ((BlockLoader.IntBuilder) builder).appendInt(((Number) v).intValue()); + } + + @Override + public String toString() { + return "SourceInts"; + } + }; + } + + /** + * Read {@code long}s from {@code _source}. + */ + public static BlockLoader longs(ValueFetcher fetcher) { + StoredFieldLoader loader = StoredFieldLoader.create(true, Set.of()); + return context -> new BlockSourceReader(fetcher, loader.getLoader(context, null)) { + @Override + public BlockLoader.Builder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.longs(expectedCount); + } + + @Override + protected void append(BlockLoader.Builder builder, Object v) { + ((BlockLoader.LongBuilder) builder).appendLong(((Number) v).longValue()); + } + + @Override + public String toString() { + return "SourceLongs"; + } + }; + } + + private final ValueFetcher fetcher; + private final LeafStoredFieldLoader loader; + private final List ignoredValues = new ArrayList<>(); + private int docID = -1; + + BlockSourceReader(ValueFetcher fetcher, LeafStoredFieldLoader loader) { + this.fetcher = fetcher; + this.loader = loader; + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) throws IOException { + BlockLoader.Builder blockBuilder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + int doc = docs.get(i); + if (doc < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + readValuesFromSingleDoc(doc, blockBuilder); + } + return blockBuilder; + } + + @Override + public void readValuesFromSingleDoc(int doc, BlockLoader.Builder builder) throws IOException { + this.docID = doc; + loader.advanceTo(doc); + List values = fetcher.fetchValues(Source.fromBytes(loader.source()), doc, ignoredValues); + ignoredValues.clear(); // TODO do something with these? + if (values == null) { + builder.appendNull(); + return; + } + if (values.size() == 1) { + append(builder, values.get(0)); + return; + } + builder.beginPositionEntry(); + for (Object v : values) { + append(builder, v); + } + builder.endPositionEntry(); + } + + protected abstract void append(BlockLoader.Builder builder, Object v); + + @Override + public int docID() { + return docID; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BlockStoredFieldsReader.java b/server/src/main/java/org/elasticsearch/index/mapper/BlockStoredFieldsReader.java new file mode 100644 index 0000000000000..d38d30a03b275 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/BlockStoredFieldsReader.java @@ -0,0 +1,152 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; +import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; +import org.elasticsearch.index.mapper.BlockLoader.BytesRefBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Set; + +/** + * Loads values from {@link LeafReader#storedFields}. This whole process is very slow + * and cast-tastic, so it doesn't really try to avoid megamorphic invocations. It's + * just going to be slow. + * + * Note that this extends {@link BlockDocValuesReader} because it pretends to load + * doc values because, for now, ESQL only knows how to load things in a doc values + * order. + */ +public abstract class BlockStoredFieldsReader extends BlockDocValuesReader { + public static BlockLoader bytesRefsFromBytesRefs(String field) { + StoredFieldLoader loader = StoredFieldLoader.create(false, Set.of(field)); + return context -> new Bytes(loader.getLoader(context, null), field) { + @Override + protected BytesRef toBytesRef(Object v) { + return (BytesRef) v; + } + }; + } + + public static BlockLoader bytesRefsFromStrings(String field) { + StoredFieldLoader loader = StoredFieldLoader.create(false, Set.of(field)); + return context -> new Bytes(loader.getLoader(context, null), field) { + private final BytesRef scratch = new BytesRef(); + + @Override + protected BytesRef toBytesRef(Object v) { + return toBytesRef(scratch, (String) v); + } + }; + } + + public static BlockLoader id() { + StoredFieldLoader loader = StoredFieldLoader.create(false, Set.of(IdFieldMapper.NAME)); + return context -> new Id(loader.getLoader(context, null)); + } + + private final LeafStoredFieldLoader loader; + private int docID = -1; + + protected BlockStoredFieldsReader(LeafStoredFieldLoader loader) { + this.loader = loader; + } + + @Override + public final BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) throws IOException { + var builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + readValuesFromSingleDoc(docs.get(i), builder); + } + return builder; + } + + @Override + public final void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) throws IOException { + if (docId < this.docID) { + throw new IllegalStateException("docs within same block must be in order"); + } + this.docID = docId; + loader.advanceTo(docId); + read(loader, builder); + } + + protected abstract void read(LeafStoredFieldLoader loader, BlockLoader.Builder builder) throws IOException; + + @Override + public final int docID() { + return docID; + } + + private abstract static class Bytes extends BlockStoredFieldsReader { + private final String field; + + Bytes(LeafStoredFieldLoader loader, String field) { + super(loader); + this.field = field; + } + + @Override + public BytesRefBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.bytesRefs(expectedCount); + } + + protected abstract BytesRef toBytesRef(Object v); + + @Override + protected void read(LeafStoredFieldLoader loader, BlockLoader.Builder builder) throws IOException { + List values = loader.storedFields().get(field); + if (values == null) { + builder.appendNull(); + return; + } + if (values.size() == 1) { + ((BytesRefBuilder) builder).appendBytesRef(toBytesRef(values.get(0))); + return; + } + builder.beginPositionEntry(); + for (Object v : values) { + ((BytesRefBuilder) builder).appendBytesRef(toBytesRef(v)); + } + builder.endPositionEntry(); + } + + @Override + public String toString() { + return "BlockStoredFieldsReader.Bytes"; + } + } + + private static class Id extends BlockStoredFieldsReader { + private final BytesRef scratch = new BytesRef(); + + Id(LeafStoredFieldLoader loader) { + super(loader); + } + + @Override + public BlockLoader.BytesRefBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.bytesRefs(expectedCount); + } + + @Override + protected void read(LeafStoredFieldLoader loader, BlockLoader.Builder builder) throws IOException { + ((BytesRefBuilder) builder).appendBytesRef(toBytesRef(scratch, loader.id())); + } + + @Override + public String toString() { + return "BlockStoredFieldsReader.Id"; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BooleanFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/BooleanFieldMapper.java index 54961f2f489bf..1fb3f706c56a2 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/BooleanFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/BooleanFieldMapper.java @@ -255,6 +255,14 @@ public Boolean valueForDisplay(Object value) { }; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (hasDocValues()) { + return BlockDocValuesReader.booleans(name()); + } + return BlockSourceReader.booleans(sourceValueFetcher(blContext.sourcePaths(name()))); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { FielddataOperation operation = fieldDataContext.fielddataOperation(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BooleanScriptBlockDocValuesReader.java b/server/src/main/java/org/elasticsearch/index/mapper/BooleanScriptBlockDocValuesReader.java new file mode 100644 index 0000000000000..0d29bc43700e8 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/BooleanScriptBlockDocValuesReader.java @@ -0,0 +1,77 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.elasticsearch.script.BooleanFieldScript; + +/** + * {@link BlockDocValuesReader} implementation for {@code boolean} scripts. + */ +public class BooleanScriptBlockDocValuesReader extends BlockDocValuesReader { + public static BlockLoader blockLoader(BooleanFieldScript.LeafFactory factory) { + return context -> new BooleanScriptBlockDocValuesReader(factory.newInstance(context)); + } + + private final BooleanFieldScript script; + private int docId; + + BooleanScriptBlockDocValuesReader(BooleanFieldScript script) { + this.script = script; + } + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.BooleanBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + // Note that we don't emit falses before trues so we conform to the doc values contract and can use booleansFromDocValues + return factory.booleansFromDocValues(expectedCount); + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.BooleanBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + read(docs.get(i), builder); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + read(docId, (BlockLoader.BooleanBuilder) builder); + } + + private void read(int docId, BlockLoader.BooleanBuilder builder) { + script.runForDoc(docId); + int total = script.falses() + script.trues(); + switch (total) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendBoolean(script.trues() > 0); + default -> { + builder.beginPositionEntry(); + for (int i = 0; i < script.falses(); i++) { + builder.appendBoolean(false); + } + for (int i = 0; i < script.trues(); i++) { + builder.appendBoolean(true); + } + builder.endPositionEntry(); + } + } + } + + @Override + public String toString() { + return "ScriptBooleans"; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/BooleanScriptFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/BooleanScriptFieldType.java index fcdf733b0febc..6e3876644567f 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/BooleanScriptFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/BooleanScriptFieldType.java @@ -110,6 +110,11 @@ public DocValueFormat docValueFormat(String format, ZoneId timeZone) { return DocValueFormat.BOOLEAN; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return BooleanScriptBlockDocValuesReader.blockLoader(leafFactory(blContext.lookup())); + } + @Override public BooleanScriptFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { return new BooleanScriptFieldData.Builder(name(), leafFactory(fieldDataContext.lookupSupplier().get()), BooleanDocValuesField::new); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java index c80df48f56977..21b9ec04c56c0 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DateFieldMapper.java @@ -773,6 +773,14 @@ public Function pointReaderIfPossible() { return null; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (hasDocValues()) { + return BlockDocValuesReader.longs(name()); + } + return BlockSourceReader.longs(sourceValueFetcher(blContext.sourcePaths(name()))); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { FielddataOperation operation = fieldDataContext.fielddataOperation(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DateScriptBlockDocValuesReader.java b/server/src/main/java/org/elasticsearch/index/mapper/DateScriptBlockDocValuesReader.java new file mode 100644 index 0000000000000..6e6cdd3d1f057 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/DateScriptBlockDocValuesReader.java @@ -0,0 +1,72 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.elasticsearch.script.DateFieldScript; + +/** + * {@link BlockDocValuesReader} implementation for date scripts. + */ +public class DateScriptBlockDocValuesReader extends BlockDocValuesReader { + public static BlockLoader blockLoader(DateFieldScript.LeafFactory factory) { + return context -> new DateScriptBlockDocValuesReader(factory.newInstance(context)); + } + + private final DateFieldScript script; + private int docId; + + DateScriptBlockDocValuesReader(DateFieldScript script) { + this.script = script; + } + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.LongBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.longs(expectedCount); // Note that we don't pre-sort our output so we can't use longsFromDocValues + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.LongBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + read(docs.get(i), builder); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + read(docId, (BlockLoader.LongBuilder) builder); + } + + private void read(int docId, BlockLoader.LongBuilder builder) { + script.runForDoc(docId); + switch (script.count()) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendLong(script.values()[0]); + default -> { + builder.beginPositionEntry(); + for (int i = 0; i < script.count(); i++) { + builder.appendLong(script.values()[i]); + } + builder.endPositionEntry(); + } + } + } + + @Override + public String toString() { + return "ScriptDates"; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DateScriptFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/DateScriptFieldType.java index 10c68e023baa2..8252d571dce68 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DateScriptFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DateScriptFieldType.java @@ -179,6 +179,11 @@ public DocValueFormat docValueFormat(@Nullable String format, ZoneId timeZone) { return new DocValueFormat.DateTime(dateTimeFormatter, timeZone, Resolution.MILLISECONDS); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return DateScriptBlockDocValuesReader.blockLoader(leafFactory(blContext.lookup())); + } + @Override public DateScriptFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { return new DateScriptFieldData.Builder( diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DoubleScriptBlockDocValuesReader.java b/server/src/main/java/org/elasticsearch/index/mapper/DoubleScriptBlockDocValuesReader.java new file mode 100644 index 0000000000000..856321f53244d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/DoubleScriptBlockDocValuesReader.java @@ -0,0 +1,72 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.elasticsearch.script.DoubleFieldScript; + +/** + * {@link BlockDocValuesReader} implementation for {@code double} scripts. + */ +public class DoubleScriptBlockDocValuesReader extends BlockDocValuesReader { + public static BlockLoader blockLoader(DoubleFieldScript.LeafFactory factory) { + return context -> new DoubleScriptBlockDocValuesReader(factory.newInstance(context)); + } + + private final DoubleFieldScript script; + private int docId; + + DoubleScriptBlockDocValuesReader(DoubleFieldScript script) { + this.script = script; + } + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.DoubleBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.doubles(expectedCount); // Note that we don't pre-sort our output so we can't use doublesFromDocValues + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.DoubleBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + read(docs.get(i), builder); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + read(docId, (BlockLoader.DoubleBuilder) builder); + } + + private void read(int docId, BlockLoader.DoubleBuilder builder) { + script.runForDoc(docId); + switch (script.count()) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendDouble(script.values()[0]); + default -> { + builder.beginPositionEntry(); + for (int i = 0; i < script.count(); i++) { + builder.appendDouble(script.values()[i]); + } + builder.endPositionEntry(); + } + } + } + + @Override + public String toString() { + return "ScriptDoubles"; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DoubleScriptFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/DoubleScriptFieldType.java index 9baa6340b45c7..ef5c112ef212a 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DoubleScriptFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DoubleScriptFieldType.java @@ -105,6 +105,11 @@ public DocValueFormat docValueFormat(String format, ZoneId timeZone) { return new DocValueFormat.Decimal(format); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return DoubleScriptBlockDocValuesReader.blockLoader(leafFactory(blContext.lookup())); + } + @Override public DoubleScriptFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { return new DoubleScriptFieldData.Builder(name(), leafFactory(fieldDataContext.lookupSupplier().get()), DoubleDocValuesField::new); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java index c24b3077f700c..cb69fa4b7c50a 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java @@ -10,6 +10,7 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.Strings; import org.elasticsearch.index.fielddata.FieldData; import org.elasticsearch.index.fielddata.FieldDataContext; @@ -77,6 +78,45 @@ public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext ); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + // TODO build a constant block directly + BytesRef bytes = new BytesRef(blContext.indexName()); + return context -> new BlockDocValuesReader() { + private int docId; + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.BytesRefBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.bytesRefs(expectedCount); + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.BytesRefBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + builder.appendBytesRef(bytes); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + ((BlockLoader.BytesRefBuilder) builder).appendBytesRef(bytes); + } + + @Override + public String toString() { + return "Index"; + } + }; + } + @Override public ValueFetcher valueFetcher(SearchExecutionContext context, String format) { return new ValueFetcher() { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java index 4f7a8d5691c69..7d6b7711360fe 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/IpFieldMapper.java @@ -406,6 +406,14 @@ public static Query rangeQuery( return builder.apply(lower, upper); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (hasDocValues()) { + return BlockDocValuesReader.bytesRefsFromOrds(name()); + } + return null; + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { failIfNoDocValues(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/IpScriptBlockDocValuesReader.java b/server/src/main/java/org/elasticsearch/index/mapper/IpScriptBlockDocValuesReader.java new file mode 100644 index 0000000000000..f05b9aff890af --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/IpScriptBlockDocValuesReader.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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.elasticsearch.script.IpFieldScript; + +/** + * {@link BlockDocValuesReader} implementation for keyword scripts. + */ +public class IpScriptBlockDocValuesReader extends BlockDocValuesReader { + public static BlockLoader blockLoader(IpFieldScript.LeafFactory factory) { + return context -> new IpScriptBlockDocValuesReader(factory.newInstance(context)); + } + + private final IpFieldScript script; + private int docId; + + IpScriptBlockDocValuesReader(IpFieldScript script) { + this.script = script; + } + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.BytesRefBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.bytesRefs(expectedCount); // Note that we don't pre-sort our output so we can't use bytesRefsFromDocValues + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.BytesRefBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + read(docs.get(i), builder); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + read(docId, (BlockLoader.BytesRefBuilder) builder); + } + + private void read(int docId, BlockLoader.BytesRefBuilder builder) { + script.runForDoc(docId); + switch (script.count()) { + case 0 -> builder.appendNull(); + case 1 -> { + builder.appendBytesRef(script.values()[0]); + } + default -> { + builder.beginPositionEntry(); + for (int i = 0; i < script.count(); i++) { + builder.appendBytesRef(script.values()[i]); + } + builder.endPositionEntry(); + } + } + } + + @Override + public String toString() { + return "ScriptIps"; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/IpScriptFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/IpScriptFieldType.java index 693322506972f..0e56b30e2d5d9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/IpScriptFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/IpScriptFieldType.java @@ -208,4 +208,9 @@ private Query cidrQuery(String term, SearchExecutionContext context) { BytesRef upperBytes = new BytesRef(InetAddressPoint.encode(InetAddressPoint.decode(upper))); return new IpScriptFieldRangeQuery(script, leafFactory(context), name(), lowerBytes, upperBytes); } + + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return IpScriptBlockDocValuesReader.blockLoader(leafFactory(blContext.lookup())); + } } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java index 1e74f90ed7393..9bc3db22365de 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/KeywordFieldMapper.java @@ -578,6 +578,24 @@ NamedAnalyzer normalizer() { return normalizer; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (hasDocValues()) { + return BlockDocValuesReader.bytesRefsFromOrds(name()); + } + if (isSyntheticSource) { + if (false == isStored()) { + throw new IllegalStateException( + "keyword field [" + + name() + + "] is only supported in synthetic _source index if it creates doc values or stored fields" + ); + } + return BlockStoredFieldsReader.bytesRefsFromBytesRefs(name()); + } + return BlockSourceReader.bytesRefs(sourceValueFetcher(blContext.sourcePaths(name()))); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { FielddataOperation operation = fieldDataContext.fielddataOperation(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/KeywordScriptBlockDocValuesReader.java b/server/src/main/java/org/elasticsearch/index/mapper/KeywordScriptBlockDocValuesReader.java new file mode 100644 index 0000000000000..51058b3b60bf4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/KeywordScriptBlockDocValuesReader.java @@ -0,0 +1,78 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.script.StringFieldScript; + +/** + * {@link BlockDocValuesReader} implementation for keyword scripts. + */ +public class KeywordScriptBlockDocValuesReader extends BlockDocValuesReader { + public static BlockLoader blockLoader(StringFieldScript.LeafFactory factory) { + return context -> new KeywordScriptBlockDocValuesReader(factory.newInstance(context)); + } + + private final BytesRefBuilder bytesBuild = new BytesRefBuilder(); + private final StringFieldScript script; + private int docId; + + KeywordScriptBlockDocValuesReader(StringFieldScript script) { + this.script = script; + } + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.BytesRefBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.bytesRefs(expectedCount); // Note that we don't pre-sort our output so we can't use bytesRefsFromDocValues + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.BytesRefBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + read(docs.get(i), builder); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + read(docId, (BlockLoader.BytesRefBuilder) builder); + } + + private void read(int docId, BlockLoader.BytesRefBuilder builder) { + script.runForDoc(docId); + switch (script.getValues().size()) { + case 0 -> builder.appendNull(); + case 1 -> { + bytesBuild.copyChars(script.getValues().get(0)); + builder.appendBytesRef(bytesBuild.get()); + } + default -> { + builder.beginPositionEntry(); + for (String v : script.getValues()) { + bytesBuild.copyChars(v); + builder.appendBytesRef(bytesBuild.get()); + } + builder.endPositionEntry(); + } + } + } + + @Override + public String toString() { + return "ScriptKeywords"; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/KeywordScriptFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/KeywordScriptFieldType.java index fb498e7eb7dcd..879a28d4c76c8 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/KeywordScriptFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/KeywordScriptFieldType.java @@ -110,6 +110,11 @@ public Object valueForDisplay(Object value) { return binaryValue.utf8ToString(); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return KeywordScriptBlockDocValuesReader.blockLoader(leafFactory(blContext.lookup())); + } + @Override public StringScriptFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { return new StringScriptFieldData.Builder(name(), leafFactory(fieldDataContext.lookupSupplier().get()), KeywordDocValuesField::new); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/LongScriptBlockDocValuesReader.java b/server/src/main/java/org/elasticsearch/index/mapper/LongScriptBlockDocValuesReader.java new file mode 100644 index 0000000000000..4896f7d858144 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/mapper/LongScriptBlockDocValuesReader.java @@ -0,0 +1,72 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.elasticsearch.script.LongFieldScript; + +/** + * {@link BlockDocValuesReader} implementation for {@code long} scripts. + */ +public class LongScriptBlockDocValuesReader extends BlockDocValuesReader { + public static BlockLoader blockLoader(LongFieldScript.LeafFactory factory) { + return context -> new LongScriptBlockDocValuesReader(factory.newInstance(context)); + } + + private final LongFieldScript script; + private int docId; + + LongScriptBlockDocValuesReader(LongFieldScript script) { + this.script = script; + } + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.LongBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.longs(expectedCount); // Note that we don't pre-sort our output so we can't use longsFromDocValues + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.LongBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + read(docs.get(i), builder); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + read(docId, (BlockLoader.LongBuilder) builder); + } + + private void read(int docId, BlockLoader.LongBuilder builder) { + script.runForDoc(docId); + switch (script.count()) { + case 0 -> builder.appendNull(); + case 1 -> builder.appendLong(script.values()[0]); + default -> { + builder.beginPositionEntry(); + for (int i = 0; i < script.count(); i++) { + builder.appendLong(script.values()[i]); + } + builder.endPositionEntry(); + } + } + } + + @Override + public String toString() { + return "ScriptLongs"; + } +} diff --git a/server/src/main/java/org/elasticsearch/index/mapper/LongScriptFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/LongScriptFieldType.java index 417d60533aad0..f89babe32d0a9 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/LongScriptFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/LongScriptFieldType.java @@ -105,6 +105,11 @@ public DocValueFormat docValueFormat(String format, ZoneId timeZone) { return new DocValueFormat.Decimal(format); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return LongScriptBlockDocValuesReader.blockLoader(leafFactory(blContext.lookup())); + } + @Override public LongScriptFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { return new LongScriptFieldData.Builder(name(), leafFactory(fieldDataContext.lookupSupplier().get()), LongDocValuesField::new); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java b/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java index 21ed56a82292c..b68bb1a2b1987 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappedFieldType.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.fetch.subphase.FetchFieldsPhase; +import org.elasticsearch.search.lookup.SearchLookup; import java.io.IOException; import java.time.ZoneId; @@ -50,6 +51,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.Function; import static org.elasticsearch.search.SearchService.ALLOW_EXPENSIVE_QUERIES; @@ -629,4 +631,33 @@ public void validateMatchedRoutingPath(String routingPath) { + "]." ); } + + /** + * Returns a loader for ESQL or {@code null} if the field doesn't support + * ESQL. + */ + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return null; + } + + /** + * Arguments for {@link #blockLoader}. + */ + public interface BlockLoaderContext { + /** + * The name of the index. + */ + String indexName(); + + /** + * {@link SearchLookup} used for building scripts. + */ + SearchLookup lookup(); + + /** + * Find the paths in {@code _source} that contain values for the field named {@code name}. + */ + Set sourcePaths(String name); + } + } diff --git a/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java index 4df3b9cf02985..7dfc5a98037d0 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/NumberFieldMapper.java @@ -438,6 +438,16 @@ protected void writeValue(XContentBuilder b, long value) throws IOException { } }; } + + @Override + BlockLoader blockLoaderFromDocValues(String fieldName) { + return BlockDocValuesReader.doubles(fieldName, l -> HalfFloatPoint.sortableShortToHalfFloat((short) l)); + } + + @Override + BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher) { + return BlockSourceReader.doubles(sourceValueFetcher); + } }, FLOAT("float", NumericType.FLOAT) { @Override @@ -590,6 +600,16 @@ protected void writeValue(XContentBuilder b, long value) throws IOException { } }; } + + @Override + BlockLoader blockLoaderFromDocValues(String fieldName) { + return BlockDocValuesReader.doubles(fieldName, l -> NumericUtils.sortableIntToFloat((int) l)); + } + + @Override + BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher) { + return BlockSourceReader.doubles(sourceValueFetcher); + } }, DOUBLE("double", NumericType.DOUBLE) { @Override @@ -720,6 +740,16 @@ protected void writeValue(XContentBuilder b, long value) throws IOException { } }; } + + @Override + BlockLoader blockLoaderFromDocValues(String fieldName) { + return BlockDocValuesReader.doubles(fieldName, NumericUtils::sortableLongToDouble); + } + + @Override + BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher) { + return BlockSourceReader.doubles(sourceValueFetcher); + } }, BYTE("byte", NumericType.BYTE) { @Override @@ -813,6 +843,16 @@ public IndexFieldData.Builder getValueFetcherFieldDataBuilder( SourceLoader.SyntheticFieldLoader syntheticFieldLoader(String fieldName, String fieldSimpleName, boolean ignoreMalformed) { return NumberType.syntheticLongFieldLoader(fieldName, fieldSimpleName, ignoreMalformed); } + + @Override + BlockLoader blockLoaderFromDocValues(String fieldName) { + return BlockDocValuesReader.ints(fieldName); + } + + @Override + BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher) { + return BlockSourceReader.ints(sourceValueFetcher); + } }, SHORT("short", NumericType.SHORT) { @Override @@ -902,6 +942,16 @@ public IndexFieldData.Builder getValueFetcherFieldDataBuilder( SourceLoader.SyntheticFieldLoader syntheticFieldLoader(String fieldName, String fieldSimpleName, boolean ignoreMalformed) { return NumberType.syntheticLongFieldLoader(fieldName, fieldSimpleName, ignoreMalformed); } + + @Override + BlockLoader blockLoaderFromDocValues(String fieldName) { + return BlockDocValuesReader.ints(fieldName); + } + + @Override + BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher) { + return BlockSourceReader.ints(sourceValueFetcher); + } }, INTEGER("integer", NumericType.INT) { @Override @@ -1059,6 +1109,16 @@ public IndexFieldData.Builder getValueFetcherFieldDataBuilder( SourceLoader.SyntheticFieldLoader syntheticFieldLoader(String fieldName, String fieldSimpleName, boolean ignoreMalformed) { return NumberType.syntheticLongFieldLoader(fieldName, fieldSimpleName, ignoreMalformed); } + + @Override + BlockLoader blockLoaderFromDocValues(String fieldName) { + return BlockDocValuesReader.ints(fieldName); + } + + @Override + BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher) { + return BlockSourceReader.ints(sourceValueFetcher); + } }, LONG("long", NumericType.LONG) { @Override @@ -1186,6 +1246,16 @@ public IndexFieldData.Builder getValueFetcherFieldDataBuilder( SourceLoader.SyntheticFieldLoader syntheticFieldLoader(String fieldName, String fieldSimpleName, boolean ignoreMalformed) { return syntheticLongFieldLoader(fieldName, fieldSimpleName, ignoreMalformed); } + + @Override + BlockLoader blockLoaderFromDocValues(String fieldName) { + return BlockDocValuesReader.longs(fieldName); + } + + @Override + BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher) { + return BlockSourceReader.longs(sourceValueFetcher); + } }; private final String name; @@ -1449,6 +1519,10 @@ protected void writeValue(XContentBuilder b, long value) throws IOException { } }; } + + abstract BlockLoader blockLoaderFromDocValues(String fieldName); + + abstract BlockLoader blockLoaderFromSource(SourceValueFetcher sourceValueFetcher); } public static class NumberFieldType extends SimpleMappedFieldType { @@ -1579,6 +1653,18 @@ public Function pointReaderIfPossible() { return null; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (indexMode == IndexMode.TIME_SERIES && metricType == TimeSeriesParams.MetricType.COUNTER) { + // Counters are not supported by ESQL so we load them in null + return BlockDocValuesReader.nulls(); + } + if (hasDocValues()) { + return type.blockLoaderFromDocValues(name()); + } + return type.blockLoaderFromSource(sourceValueFetcher(blContext.sourcePaths(name()))); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { FielddataOperation operation = fieldDataContext.fielddataOperation(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ProvidedIdFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/ProvidedIdFieldMapper.java index e2821189f0564..f681d54ebbead 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ProvidedIdFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ProvidedIdFieldMapper.java @@ -117,6 +117,11 @@ public Query termsQuery(Collection values, SearchExecutionContext context) { return new TermInSetQuery(name(), bytesRefs); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return BlockStoredFieldsReader.id(); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { if (fieldDataEnabled.getAsBoolean() == false) { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java index 7ac524a642401..91616041f65f6 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/TextFieldMapper.java @@ -936,6 +936,30 @@ public boolean isAggregatable() { return fielddata; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (syntheticSourceDelegate != null) { + return syntheticSourceDelegate.blockLoader(blContext); + } + if (isSyntheticSource) { + if (isStored()) { + return BlockStoredFieldsReader.bytesRefsFromStrings(name()); + } + /* + * We *shouldn't fall to this exception. The mapping should be + * rejected because we've enabled synthetic source but not configured + * the index properly. But we give it a nice message anyway just in + * case. + */ + throw new IllegalArgumentException( + "fetching values from a text field [" + + name() + + "] is supported because synthetic _source is enabled and we don't have a way to load the fields" + ); + } + return BlockSourceReader.bytesRefs(SourceValueFetcher.toString(blContext.sourcePaths(name()))); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { FielddataOperation operation = fieldDataContext.fielddataOperation(); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/TsidExtractingIdFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/TsidExtractingIdFieldMapper.java index e8e4b6909fc4e..9d43ef398feac 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/TsidExtractingIdFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/TsidExtractingIdFieldMapper.java @@ -87,6 +87,11 @@ public Query termsQuery(Collection values, SearchExecutionContext context) { return new TermInSetQuery(name(), bytesRefs); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return BlockStoredFieldsReader.id(); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { throw new IllegalArgumentException("Fielddata is not supported on [_id] field in [time_series] indices"); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java index 0de2a27fbaac2..54a44dd55caa4 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/VersionFieldMapper.java @@ -54,6 +54,11 @@ public ValueFetcher valueFetcher(SearchExecutionContext context, String format) return new DocValueFetcher(docValueFormat(format, null), context.getForField(this, FielddataOperation.SEARCH)); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + return BlockDocValuesReader.longs(name()); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { failIfNoDocValues(); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/BooleanFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/BooleanFieldMapperTests.java index e8c2db2ab2616..70e2fee7a003a 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/BooleanFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/BooleanFieldMapperTests.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.util.List; +import java.util.function.Function; import static org.hamcrest.Matchers.equalTo; @@ -262,6 +263,12 @@ public List invalidExample() throws IOException { }; } + @Override + protected Function loadBlockExpected() { + // Just assert that we expect a boolean. Otherwise no munging. + return v -> (Boolean) v; + } + protected IngestScriptSupport ingestScriptSupport() { return new IngestScriptSupport() { @Override diff --git a/server/src/test/java/org/elasticsearch/index/mapper/BooleanScriptFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/BooleanScriptFieldTypeTests.java index 3df28170938e1..8d5a47f08c663 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/BooleanScriptFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/BooleanScriptFieldTypeTests.java @@ -410,6 +410,19 @@ public XContentParser parser() { } } + public void testBlockLoader() throws IOException { + try (Directory directory = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [false]}")))); + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [true]}")))); + try (DirectoryReader reader = iw.getReader()) { + BooleanScriptFieldType fieldType = build("xor_param", Map.of("param", false), OnScriptError.FAIL); + List expected = List.of(false, true); + assertThat(blockLoaderReadValues(reader, fieldType), equalTo(expected)); + assertThat(blockLoaderReadValuesFromSingleDoc(reader, fieldType), equalTo(expected)); + } + } + } + private void assertSameCount(IndexSearcher searcher, String source, Object queryDescription, Query scriptedQuery, Query ootbQuery) throws IOException { assertThat( diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java index 638af8562a098..9e9437aa6b9db 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldMapperTests.java @@ -34,6 +34,7 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.List; +import java.util.function.Function; import static org.elasticsearch.index.mapper.DateFieldMapper.DEFAULT_DATE_TIME_FORMATTER; import static org.hamcrest.Matchers.containsString; @@ -577,7 +578,12 @@ protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) public SyntheticSourceExample example(int maxValues) { if (randomBoolean()) { Tuple v = generateValue(); - return new SyntheticSourceExample(v.v1(), v.v2(), this::mapping); + return new SyntheticSourceExample( + v.v1(), + v.v2(), + resolution.convert(Instant.from(formatter.parse(v.v2()))), + this::mapping + ); } List> values = randomList(1, maxValues, this::generateValue); List in = values.stream().map(Tuple::v1).toList(); @@ -588,7 +594,10 @@ public SyntheticSourceExample example(int maxValues) { .map(Tuple::v2) .toList(); Object out = outList.size() == 1 ? outList.get(0) : outList; - return new SyntheticSourceExample(in, out, this::mapping); + + List outBlockList = outList.stream().map(v -> resolution.convert(Instant.from(formatter.parse(v)))).toList(); + Object outBlock = outBlockList.size() == 1 ? outBlockList.get(0) : outBlockList; + return new SyntheticSourceExample(in, out, outBlock, this::mapping); } private Tuple generateValue() { @@ -696,6 +705,11 @@ public void execute() { }; } + @Override + protected Function loadBlockExpected() { + return v -> ((Number) v).longValue(); + } + public void testLegacyField() throws Exception { // check that unknown date formats are treated leniently on old indices MapperService service = createMapperService(IndexVersion.fromId(5000099), Settings.EMPTY, () -> false, mapping(b -> { diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DateScriptFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DateScriptFieldTypeTests.java index cf28b46118d65..d1652b9f57716 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DateScriptFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DateScriptFieldTypeTests.java @@ -471,6 +471,18 @@ public void testLegacyDateFormatName() throws IOException { ); } + public void testBlockLoader() throws IOException { + try (Directory directory = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"timestamp\": [1595432181354]}")))); + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"timestamp\": [1595432181355]}")))); + try (DirectoryReader reader = iw.getReader()) { + DateScriptFieldType fieldType = build("add_days", Map.of("days", 1), OnScriptError.FAIL); + assertThat(blockLoaderReadValues(reader, fieldType), equalTo(List.of(1595518581354L, 1595518581355L))); + assertThat(blockLoaderReadValuesFromSingleDoc(reader, fieldType), equalTo(List.of(1595518581354L, 1595518581355L))); + } + } + } + @Override protected Query randomTermsQuery(MappedFieldType ft, SearchExecutionContext ctx) { return ft.termsQuery(randomList(1, 100, DateScriptFieldTypeTests::randomDate), ctx); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DoubleScriptFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DoubleScriptFieldTypeTests.java index d360dc9796d89..0f05dad8098f4 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DoubleScriptFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DoubleScriptFieldTypeTests.java @@ -230,6 +230,18 @@ public void testTermsQuery() throws IOException { } } + public void testBlockLoader() throws IOException { + try (Directory directory = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [1]}")))); + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [2]}")))); + try (DirectoryReader reader = iw.getReader()) { + DoubleScriptFieldType fieldType = build("add_param", Map.of("param", 1), OnScriptError.FAIL); + assertThat(blockLoaderReadValues(reader, fieldType), equalTo(List.of(2d, 3d))); + assertThat(blockLoaderReadValuesFromSingleDoc(reader, fieldType), equalTo(List.of(2d, 3d))); + } + } + } + @Override protected Query randomTermsQuery(MappedFieldType ft, SearchExecutionContext ctx) { return ft.termsQuery(List.of(randomLong()), ctx); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FloatFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FloatFieldMapperTests.java index 36bb70210a439..3798129ccff29 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FloatFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FloatFieldMapperTests.java @@ -14,6 +14,7 @@ import java.io.IOException; import java.util.List; +import java.util.function.Function; public class FloatFieldMapperTests extends NumberFieldMapperTests { @@ -54,6 +55,15 @@ protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) return new NumberSyntheticSourceSupport(Number::floatValue, ignoreMalformed); } + @Override + protected Function loadBlockExpected() { + return v -> { + // The test converts the float into a string so we do do + Number n = (Number) v; + return Double.parseDouble(Float.toString(n.floatValue())); + }; + } + @Override protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/HalfFloatFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/HalfFloatFieldMapperTests.java index 13b6644520745..cc024efb5f307 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/HalfFloatFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/HalfFloatFieldMapperTests.java @@ -16,6 +16,7 @@ import java.io.IOException; import java.util.List; +import java.util.function.Function; public class HalfFloatFieldMapperTests extends NumberFieldMapperTests { @@ -54,6 +55,17 @@ protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) ); } + @Override + protected Function loadBlockExpected() { + return v -> { + // The test converts the float into a string so we do do + Number n = (Number) v; + return Double.parseDouble( + Float.toString(HalfFloatPoint.sortableShortToHalfFloat(HalfFloatPoint.halfFloatToSortableShort(n.floatValue()))) + ); + }; + } + @Override protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/IpFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/IpFieldMapperTests.java index d2510d3671d23..ba9c2e6c4a299 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/IpFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/IpFieldMapperTests.java @@ -28,6 +28,7 @@ import java.net.InetAddress; import java.util.ArrayList; import java.util.List; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -386,6 +387,7 @@ public List invalidExample() throws IOException { } } + @Override protected IngestScriptSupport ingestScriptSupport() { return new IngestScriptSupport() { @Override @@ -419,4 +421,9 @@ public void execute() { } }; } + + @Override + protected Function loadBlockExpected() { + return v -> InetAddresses.toAddrString(InetAddressPoint.decode(BytesRef.deepCopyOf((BytesRef) v).bytes)); + } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/IpScriptFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/IpScriptFieldTypeTests.java index 26e1763a12c21..56ca5f3dae89f 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/IpScriptFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/IpScriptFieldTypeTests.java @@ -8,6 +8,7 @@ package org.elasticsearch.index.mapper; +import org.apache.lucene.document.InetAddressPoint; import org.apache.lucene.document.StoredField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.LeafReaderContext; @@ -25,6 +26,7 @@ import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.lucene.search.function.ScriptScoreQuery; +import org.elasticsearch.common.network.InetAddresses; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.fielddata.BinaryScriptFieldData; import org.elasticsearch.index.fielddata.ScriptDocValues.Strings; @@ -244,6 +246,22 @@ public void testTermsQuery() throws IOException { } } + public void testBlockLoader() throws IOException { + try (Directory directory = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [\"192.168.0\"]}")))); + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [\"192.168.1\"]}")))); + try (DirectoryReader reader = iw.getReader()) { + IpScriptFieldType fieldType = build("append_param", Map.of("param", ".1"), OnScriptError.FAIL); + List expected = List.of( + new BytesRef(InetAddressPoint.encode(InetAddresses.forString("192.168.0.1"))), + new BytesRef(InetAddressPoint.encode(InetAddresses.forString("192.168.1.1"))) + ); + assertThat(blockLoaderReadValues(reader, fieldType), equalTo(expected)); + assertThat(blockLoaderReadValuesFromSingleDoc(reader, fieldType), equalTo(expected)); + } + } + } + @Override protected Query randomTermsQuery(MappedFieldType ft, SearchExecutionContext ctx) { return ft.termsQuery(randomList(100, () -> randomIp(randomBoolean())), ctx); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java index 94e2506d2b2a7..eafb33cd44cd4 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/KeywordFieldMapperTests.java @@ -50,6 +50,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; import static java.util.Collections.singletonList; @@ -648,6 +649,11 @@ protected boolean supportsIgnoreMalformed() { return false; } + @Override + protected Function loadBlockExpected() { + return v -> ((BytesRef) v).utf8ToString(); + } + @Override protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) { assertFalse("keyword doesn't support ignore_malformed", ignoreMalformed); @@ -658,6 +664,7 @@ static class KeywordSyntheticSourceSupport implements SyntheticSourceSupport { private final Integer ignoreAbove = randomBoolean() ? null : between(10, 100); private final boolean allIgnored = ignoreAbove != null && rarely(); private final boolean store; + private final boolean docValues; private final String nullValue; private final boolean exampleSortsUsingIgnoreAbove; @@ -665,13 +672,18 @@ static class KeywordSyntheticSourceSupport implements SyntheticSourceSupport { this.store = store; this.nullValue = nullValue; this.exampleSortsUsingIgnoreAbove = exampleSortsUsingIgnoreAbove; + this.docValues = store ? randomBoolean() : true; } @Override public SyntheticSourceExample example(int maxValues) { if (randomBoolean()) { Tuple v = generateValue(); - return new SyntheticSourceExample(v.v1(), v.v2(), this::mapping); + Object loadBlock = v.v2(); + if (ignoreAbove != null && v.v2().length() > ignoreAbove) { + loadBlock = null; + } + return new SyntheticSourceExample(v.v1(), v.v2(), loadBlock, this::mapping); } List> values = randomList(1, maxValues, this::generateValue); List in = values.stream().map(Tuple::v1).toList(); @@ -685,9 +697,13 @@ public SyntheticSourceExample example(int maxValues) { } }); List outList = store ? outPrimary : new HashSet<>(outPrimary).stream().sorted().collect(Collectors.toList()); + List loadBlock = docValues + ? new HashSet<>(outPrimary).stream().sorted().collect(Collectors.toList()) + : List.copyOf(outList); + Object loadBlockResult = loadBlock.size() == 1 ? loadBlock.get(0) : loadBlock; outList.addAll(outExtraValues); Object out = outList.size() == 1 ? outList.get(0) : outList; - return new SyntheticSourceExample(in, out, this::mapping); + return new SyntheticSourceExample(in, out, loadBlockResult, this::mapping); } private Tuple generateValue() { @@ -712,9 +728,9 @@ private void mapping(XContentBuilder b) throws IOException { } if (store) { b.field("store", true); - if (randomBoolean()) { - b.field("doc_values", false); - } + } + if (docValues == false) { + b.field("doc_values", false); } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/KeywordScriptFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/KeywordScriptFieldTypeTests.java index c319ac51803db..65f4c2e3ea6eb 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/KeywordScriptFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/KeywordScriptFieldTypeTests.java @@ -376,6 +376,21 @@ public void testMatchQuery() throws IOException { } } + public void testBlockLoader() throws IOException { + try (Directory directory = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [1]}")))); + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [2]}")))); + try (DirectoryReader reader = iw.getReader()) { + KeywordScriptFieldType fieldType = build("append_param", Map.of("param", "-Suffix"), OnScriptError.FAIL); + assertThat(blockLoaderReadValues(reader, fieldType), equalTo(List.of(new BytesRef("1-Suffix"), new BytesRef("2-Suffix")))); + assertThat( + blockLoaderReadValuesFromSingleDoc(reader, fieldType), + equalTo(List.of(new BytesRef("1-Suffix"), new BytesRef("2-Suffix"))) + ); + } + } + } + @Override protected KeywordScriptFieldType simpleMappedFieldType() { return build("read_foo", Map.of(), OnScriptError.FAIL); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/LongFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/LongFieldMapperTests.java index ad34c407d0678..f2d4431e5c79f 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/LongFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/LongFieldMapperTests.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.math.BigInteger; import java.util.List; +import java.util.function.Function; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @@ -120,6 +121,17 @@ public void testFetchCoerced() throws IOException { assertFetch(randomFetchTestMapper(), "field", 3.783147882954537E18, randomFetchTestFormat()); } + @Override + protected Function loadBlockExpected() { + return n -> { + Number number = ((Number) n); + if (Integer.MIN_VALUE <= number.longValue() && number.longValue() <= Integer.MAX_VALUE) { + return number.intValue(); + } + return number.longValue(); + }; + } + protected IngestScriptSupport ingestScriptSupport() { return new IngestScriptSupport() { @Override diff --git a/server/src/test/java/org/elasticsearch/index/mapper/LongScriptFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/LongScriptFieldTypeTests.java index 20ae732f9c5b0..1688cab24af3e 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/LongScriptFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/LongScriptFieldTypeTests.java @@ -263,6 +263,18 @@ public void testTermsQuery() throws IOException { } } + public void testBlockLoader() throws IOException { + try (Directory directory = newDirectory(); RandomIndexWriter iw = new RandomIndexWriter(random(), directory)) { + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [1]}")))); + iw.addDocument(List.of(new StoredField("_source", new BytesRef("{\"foo\": [2]}")))); + try (DirectoryReader reader = iw.getReader()) { + LongScriptFieldType fieldType = build("add_param", Map.of("param", 1), OnScriptError.FAIL); + assertThat(blockLoaderReadValues(reader, fieldType), equalTo(List.of(2L, 3L))); + assertThat(blockLoaderReadValuesFromSingleDoc(reader, fieldType), equalTo(List.of(2L, 3L))); + } + } + } + @Override protected Query randomTermsQuery(MappedFieldType ft, SearchExecutionContext ctx) { return ft.termsQuery(List.of(randomLong()), ctx); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java index 45a1ac2ced32d..7b91c84a05c53 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldMapperTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.script.ScriptFactory; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.XContentBuilder; +import org.hamcrest.Matcher; import java.io.IOException; import java.util.ArrayList; @@ -37,6 +38,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.matchesPattern; +import static org.hamcrest.Matchers.notANumber; public abstract class NumberFieldMapperTests extends MapperTestCase { @@ -379,6 +381,16 @@ public void testAllowMultipleValuesField() throws IOException { assertThat(e.getCause().getMessage(), containsString("Only one field can be stored per key")); } + @Override + protected Function loadBlockExpected() { + return n -> ((Number) n); // Just assert it's a number + } + + @Override + protected Matcher blockItemMatcher(Object expected) { + return "NaN".equals(expected) ? notANumber() : equalTo(expected); + } + protected abstract Number randomNumber(); protected final class NumberSyntheticSourceSupport implements SyntheticSourceSupport { @@ -398,10 +410,11 @@ public SyntheticSourceExample example(int maxVals) { if (randomBoolean()) { Tuple v = generateValue(); if (v.v2() instanceof Number n) { - return new SyntheticSourceExample(v.v1(), round.apply(n), this::mapping); + Number result = round.apply(n); + return new SyntheticSourceExample(v.v1(), result, result, this::mapping); } // ignore_malformed value - return new SyntheticSourceExample(v.v1(), v.v2(), this::mapping); + return new SyntheticSourceExample(v.v1(), v.v2(), List.of(), this::mapping); } List> values = randomList(1, maxVals, this::generateValue); List in = values.stream().map(Tuple::v1).toList(); @@ -412,7 +425,14 @@ public SyntheticSourceExample example(int maxVals) { .collect(Collectors.toCollection(ArrayList::new)); values.stream().filter(v -> false == v.v2() instanceof Number).map(v -> v.v2()).forEach(outList::add); Object out = outList.size() == 1 ? outList.get(0) : outList; - return new SyntheticSourceExample(in, out, this::mapping); + + List outBlockList = values.stream() + .filter(v -> v.v2() instanceof Number) + .map(t -> round.apply((Number) t.v2())) + .sorted() + .collect(Collectors.toCollection(ArrayList::new)); + Object outBlock = outBlockList.size() == 1 ? outBlockList.get(0) : outBlockList; + return new SyntheticSourceExample(in, out, outBlock, this::mapping); } private Tuple generateValue() { diff --git a/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java b/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java index 02259a24a5e94..bbfeaaa8b9d69 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/TextFieldMapperTests.java @@ -82,6 +82,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -1132,10 +1133,11 @@ public SyntheticSourceExample example(int maxValues) { return new SyntheticSourceExample( delegate.inputValue(), delegate.result(), + delegate.result(), b -> b.field("type", "text").field("store", true) ); } - return new SyntheticSourceExample(delegate.inputValue(), delegate.result(), b -> { + return new SyntheticSourceExample(delegate.inputValue(), delegate.result(), delegate.blockLoaderResult(), b -> { b.field("type", "text"); b.startObject("fields"); { @@ -1181,6 +1183,11 @@ public List invalidExample() throws IOException { }; } + @Override + protected Function loadBlockExpected() { + return v -> ((BytesRef) v).utf8ToString(); + } + @Override protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/AbstractScriptFieldTypeTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/AbstractScriptFieldTypeTestCase.java index 6b56418e14663..56ad35bee83d5 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/AbstractScriptFieldTypeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/AbstractScriptFieldTypeTestCase.java @@ -11,6 +11,7 @@ import org.apache.lucene.document.StoredField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; @@ -37,6 +38,7 @@ import org.elasticsearch.xcontent.json.JsonXContent; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -381,6 +383,53 @@ public final void testCacheable() throws IOException { } } + protected final List blockLoaderReadValues(DirectoryReader reader, MappedFieldType fieldType) throws IOException { + BlockLoader loader = fieldType.blockLoader(blContext()); + List all = new ArrayList<>(); + for (LeafReaderContext ctx : reader.leaves()) { + TestBlock block = (TestBlock) loader.reader(ctx).readValues(TestBlock.FACTORY, TestBlock.docs(ctx)); + for (int i = 0; i < block.size(); i++) { + all.add(block.get(i)); + } + } + return all; + } + + protected final List blockLoaderReadValuesFromSingleDoc(DirectoryReader reader, MappedFieldType fieldType) throws IOException { + BlockLoader loader = fieldType.blockLoader(blContext()); + List all = new ArrayList<>(); + for (LeafReaderContext ctx : reader.leaves()) { + BlockDocValuesReader blockReader = loader.reader(ctx); + TestBlock block = (TestBlock) blockReader.builder(TestBlock.FACTORY, ctx.reader().numDocs()); + for (int i = 0; i < ctx.reader().numDocs(); i++) { + blockReader.readValuesFromSingleDoc(i, block); + } + for (int i = 0; i < block.size(); i++) { + all.add(block.get(i)); + } + } + return all; + } + + private MappedFieldType.BlockLoaderContext blContext() { + return new MappedFieldType.BlockLoaderContext() { + @Override + public String indexName() { + throw new UnsupportedOperationException(); + } + + @Override + public SearchLookup lookup() { + return mockContext().lookup(); + } + + @Override + public Set sourcePaths(String name) { + throw new UnsupportedOperationException(); + } + }; + } + private void assertQueryOnlyOnText(String queryName, ThrowingRunnable buildQuery) { Exception e = expectThrows(IllegalArgumentException.class, buildQuery); assertThat( diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java index 9c9d1d763fbd1..e34072fbf1668 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperTestCase.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.IndexVersions; @@ -52,6 +53,7 @@ import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.search.lookup.Source; import org.elasticsearch.search.lookup.SourceProvider; +import org.elasticsearch.test.ListMatcher; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.json.JsonXContent; @@ -71,6 +73,7 @@ import java.util.stream.IntStream; import static java.util.stream.Collectors.toList; +import static org.elasticsearch.test.MapMatcher.assertMap; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -1034,10 +1037,24 @@ protected String minimalIsInvalidRoutingPathErrorMessage(Mapper mapper) { public record SyntheticSourceExample( CheckedConsumer inputValue, CheckedConsumer result, + CheckedConsumer blockLoaderResult, CheckedConsumer mapping ) { public SyntheticSourceExample(Object inputValue, Object result, CheckedConsumer mapping) { - this(b -> b.value(inputValue), b -> b.value(result), mapping); + this(b -> b.value(inputValue), b -> b.value(result), b -> b.value(result), mapping); + } + + /** + * Create an example that returns different results from doc values + * than from synthetic source. + */ + public SyntheticSourceExample( + Object inputValue, + Object result, + Object blockLoaderResults, + CheckedConsumer mapping + ) { + this(b -> b.value(inputValue), b -> b.value(result), b -> b.value(blockLoaderResults), mapping); } private void buildInput(XContentBuilder b) throws IOException { @@ -1050,6 +1067,20 @@ private String expected() throws IOException { result.accept(b); return Strings.toString(b.endObject()); } + + private Object expectedParsed() throws IOException { + return XContentHelper.convertToMap(JsonXContent.jsonXContent, expected(), false).get("field"); + } + + private String expectedBlockLoader() throws IOException { + XContentBuilder b = JsonXContent.contentBuilder().startObject().field("field"); + blockLoaderResult.accept(b); + return Strings.toString(b.endObject()); + } + + private Object expectedParsedBlockLoader() throws IOException { + return XContentHelper.convertToMap(JsonXContent.jsonXContent, expectedBlockLoader(), false).get("field"); + } } public record SyntheticSourceInvalidExample(Matcher error, CheckedConsumer mapping) {} @@ -1078,7 +1109,7 @@ public final void testSyntheticSourceIgnoreMalformedExamples() throws IOExceptio assumeTrue("type doesn't support ignore_malformed", supportsIgnoreMalformed()); CheckedConsumer mapping = syntheticSourceSupport(true).example(1).mapping(); for (ExampleMalformedValue v : exampleMalformedValues()) { - assertSyntheticSource(new SyntheticSourceExample(v.value, v.value, mapping)); + assertSyntheticSource(new SyntheticSourceExample(v.value, v.value, v.value, mapping)); } } @@ -1209,6 +1240,99 @@ public final void testSyntheticEmptyListNoDocValuesLoader() throws IOException { assertNoDocValueLoader(b -> b.startArray("field").endArray()); } + public final void testBlockLoaderReadValues() throws IOException { + testBlockLoader(blockReader -> (TestBlock) blockReader.readValues(TestBlock.FACTORY, TestBlock.docs(0))); + } + + public final void testBlockLoaderReadValuesFromSingleDoc() throws IOException { + testBlockLoader(blockReader -> { + TestBlock block = (TestBlock) blockReader.builder(TestBlock.FACTORY, 1); + blockReader.readValuesFromSingleDoc(0, block); + return block; + }); + } + + private void testBlockLoader(CheckedFunction body) throws IOException { + SyntheticSourceExample example = syntheticSourceSupport(false).example(5); + MapperService mapper = createMapperService(syntheticSourceMapping(b -> { + b.startObject("field"); + example.mapping().accept(b); + b.endObject(); + })); + BlockLoader loader = mapper.fieldType("field").blockLoader(new MappedFieldType.BlockLoaderContext() { + @Override + public String indexName() { + throw new UnsupportedOperationException(); + } + + @Override + public SearchLookup lookup() { + throw new UnsupportedOperationException(); + } + + @Override + public Set sourcePaths(String name) { + return mapper.mappingLookup().sourcePaths(name); + } + }); + Function valuesConvert = loadBlockExpected(); + if (valuesConvert == null) { + assertNull(loader); + return; + } + try (Directory directory = newDirectory()) { + RandomIndexWriter iw = new RandomIndexWriter(random(), directory); + LuceneDocument doc = mapper.documentMapper().parse(source(b -> { + b.field("field"); + example.inputValue.accept(b); + })).rootDoc(); + iw.addDocument(doc); + iw.close(); + try (DirectoryReader reader = DirectoryReader.open(directory)) { + TestBlock block = body.apply(loader.reader(reader.leaves().get(0))); + Object inBlock = block.get(0); + if (inBlock != null) { + if (inBlock instanceof List l) { + inBlock = l.stream().map(valuesConvert).toList(); + } else { + inBlock = valuesConvert.apply(inBlock); + } + } + Object expected = loader instanceof BlockSourceReader ? example.expectedParsed() : example.expectedParsedBlockLoader(); + if (List.of().equals(expected)) { + assertThat(inBlock, nullValue()); + return; + } + if (expected instanceof List l) { + ListMatcher m = ListMatcher.matchesList(); + for (Object v : l) { + m = m.item(blockItemMatcher(v)); + } + assertMap((List) inBlock, m); + return; + } + @SuppressWarnings("unchecked") + Matcher e = (Matcher) blockItemMatcher(expected); + assertThat(inBlock, e); + } + } + } + + /** + * Matcher for {@link #testBlockLoaderReadValues} and {@link #testBlockLoaderReadValuesFromSingleDoc}. + */ + protected Matcher blockItemMatcher(Object expected) { + return equalTo(expected); + } + + /** + * How {@link MappedFieldType#blockLoader} should load values or {@code null} + * if that method isn't supported by field being tested. + */ + protected Function loadBlockExpected() { + return null; + } + public final void testEmptyDocumentNoDocValueLoader() throws IOException { assumeFalse("Field will add values even if no fields are supplied", addsValueWhenNotSupplied()); assertNoDocValueLoader(b -> {}); 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 new file mode 100644 index 0000000000000..5a42d5c6890b5 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/TestBlock.java @@ -0,0 +1,199 @@ +/* + * 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 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.mapper; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.core.Nullable; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +public class TestBlock + implements + BlockLoader.BooleanBuilder, + BlockLoader.BytesRefBuilder, + BlockLoader.DoubleBuilder, + BlockLoader.IntBuilder, + BlockLoader.LongBuilder, + BlockLoader.SingletonOrdinalsBuilder { + public static BlockLoader.BuilderFactory FACTORY = new BlockLoader.BuilderFactory() { + @Override + public BlockLoader.BooleanBuilder booleansFromDocValues(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.BooleanBuilder booleans(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.BytesRefBuilder bytesRefsFromDocValues(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.BytesRefBuilder bytesRefs(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.DoubleBuilder doublesFromDocValues(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.DoubleBuilder doubles(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.IntBuilder intsFromDocValues(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.IntBuilder ints(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.LongBuilder longsFromDocValues(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.LongBuilder longs(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.Builder nulls(int expectedCount) { + return new TestBlock(null); + } + + @Override + public BlockLoader.SingletonOrdinalsBuilder singletonOrdinalsBuilder(SortedDocValues ordinals, int count) { + return new TestBlock(ordinals); + } + }; + + public static final BlockLoader.Docs docs(int... docs) { + return new BlockLoader.Docs() { + @Override + public int count() { + return docs.length; + } + + @Override + public int get(int i) { + return docs[i]; + } + }; + } + + public static final BlockLoader.Docs docs(LeafReaderContext ctx) { + return new BlockLoader.Docs() { + @Override + public int count() { + return ctx.reader().numDocs(); + } + + @Override + public int get(int i) { + return i; + } + }; + } + + private final SortedDocValues sortedDocValues; + private final List values = new ArrayList<>(); + + private List currentPosition = null; + + private TestBlock(@Nullable SortedDocValues sortedDocValues) { + this.sortedDocValues = sortedDocValues; + } + + public Object get(int i) { + return values.get(i); + } + + public int size() { + return values.size(); + } + + @Override + public TestBlock appendNull() { + assertNull(currentPosition); + values.add(null); + return this; + } + + @Override + public TestBlock beginPositionEntry() { + assertNull(currentPosition); + currentPosition = new ArrayList<>(); + values.add(currentPosition); + return this; + } + + @Override + public TestBlock endPositionEntry() { + assertNotNull(currentPosition); + currentPosition = null; + return this; + } + + @Override + public TestBlock appendBoolean(boolean value) { + return add(value); + } + + @Override + public TestBlock appendBytesRef(BytesRef value) { + return add(BytesRef.deepCopyOf(value)); + } + + @Override + public TestBlock appendDouble(double value) { + return add(value); + } + + @Override + public TestBlock appendInt(int value) { + return add(value); + } + + @Override + public TestBlock appendLong(long value) { + return add(value); + } + + @Override + public TestBlock appendOrd(int value) { + try { + return add(sortedDocValues.lookupOrd(value)); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private TestBlock add(Object value) { + (currentPosition == null ? values : currentPosition).add(value); + return this; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlock.java index f8f291c2a2e69..c518b3d603319 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BooleanBlock.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.BlockLoader; import java.io.IOException; @@ -185,11 +186,14 @@ static BooleanBlock newConstantBlockWith(boolean value, int positions, BlockFact return blockFactory.newConstantBooleanBlockWith(value, positions); } - sealed interface Builder extends Block.Builder permits BooleanBlockBuilder { - + /** + * Builder for {@link BooleanBlock} + */ + sealed interface Builder extends Block.Builder, BlockLoader.BooleanBuilder permits BooleanBlockBuilder { /** * Appends a boolean to the current entry. */ + @Override Builder appendBoolean(boolean value); /** @@ -213,12 +217,11 @@ sealed interface Builder extends Block.Builder permits BooleanBlockBuilder { @Override Builder mvOrdering(Block.MvOrdering mvOrdering); - // TODO boolean containsMvDups(); - /** * Appends the all values of the given block into a the current position * in this builder. */ + @Override Builder appendAllValuesToCurrentPosition(Block block); /** diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlock.java index 488d3032b2b08..6da60fbfe011d 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/BytesRefBlock.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.BlockLoader; import java.io.IOException; @@ -190,11 +191,14 @@ static BytesRefBlock newConstantBlockWith(BytesRef value, int positions, BlockFa return blockFactory.newConstantBytesRefBlockWith(value, positions); } - sealed interface Builder extends Block.Builder permits BytesRefBlockBuilder { - + /** + * Builder for {@link BytesRefBlock} + */ + sealed interface Builder extends Block.Builder, BlockLoader.BytesRefBuilder permits BytesRefBlockBuilder { /** * Appends a BytesRef to the current entry. */ + @Override Builder appendBytesRef(BytesRef value); /** @@ -218,12 +222,11 @@ sealed interface Builder extends Block.Builder permits BytesRefBlockBuilder { @Override Builder mvOrdering(Block.MvOrdering mvOrdering); - // TODO boolean containsMvDups(); - /** * Appends the all values of the given block into a the current position * in this builder. */ + @Override Builder appendAllValuesToCurrentPosition(Block block); /** diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlock.java index c2e63a0c6f384..9d42e5dd3f284 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/DoubleBlock.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.BlockLoader; import java.io.IOException; @@ -186,11 +187,14 @@ static DoubleBlock newConstantBlockWith(double value, int positions, BlockFactor return blockFactory.newConstantDoubleBlockWith(value, positions); } - sealed interface Builder extends Block.Builder permits DoubleBlockBuilder { - + /** + * Builder for {@link DoubleBlock} + */ + sealed interface Builder extends Block.Builder, BlockLoader.DoubleBuilder permits DoubleBlockBuilder { /** * Appends a double to the current entry. */ + @Override Builder appendDouble(double value); /** @@ -214,12 +218,11 @@ sealed interface Builder extends Block.Builder permits DoubleBlockBuilder { @Override Builder mvOrdering(Block.MvOrdering mvOrdering); - // TODO boolean containsMvDups(); - /** * Appends the all values of the given block into a the current position * in this builder. */ + @Override Builder appendAllValuesToCurrentPosition(Block block); /** diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java index f27e855809491..d4305e23dd4a1 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/IntBlock.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.BlockLoader; import java.io.IOException; @@ -185,11 +186,14 @@ static IntBlock newConstantBlockWith(int value, int positions, BlockFactory bloc return blockFactory.newConstantIntBlockWith(value, positions); } - sealed interface Builder extends Block.Builder permits IntBlockBuilder { - + /** + * Builder for {@link IntBlock} + */ + sealed interface Builder extends Block.Builder, BlockLoader.IntBuilder permits IntBlockBuilder { /** * Appends a int to the current entry. */ + @Override Builder appendInt(int value); /** @@ -213,12 +217,11 @@ sealed interface Builder extends Block.Builder permits IntBlockBuilder { @Override Builder mvOrdering(Block.MvOrdering mvOrdering); - // TODO boolean containsMvDups(); - /** * Appends the all values of the given block into a the current position * in this builder. */ + @Override Builder appendAllValuesToCurrentPosition(Block block); /** diff --git a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlock.java b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlock.java index 287b55eac3d04..9def8475161ff 100644 --- a/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlock.java +++ b/x-pack/plugin/esql/compute/src/main/generated-src/org/elasticsearch/compute/data/LongBlock.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.BlockLoader; import java.io.IOException; @@ -186,11 +187,14 @@ static LongBlock newConstantBlockWith(long value, int positions, BlockFactory bl return blockFactory.newConstantLongBlockWith(value, positions); } - sealed interface Builder extends Block.Builder permits LongBlockBuilder { - + /** + * Builder for {@link LongBlock} + */ + sealed interface Builder extends Block.Builder, BlockLoader.LongBuilder permits LongBlockBuilder { /** * Appends a long to the current entry. */ + @Override Builder appendLong(long value); /** @@ -214,12 +218,11 @@ sealed interface Builder extends Block.Builder permits LongBlockBuilder { @Override Builder mvOrdering(Block.MvOrdering mvOrdering); - // TODO boolean containsMvDups(); - /** * Appends the all values of the given block into a the current position * in this builder. */ + @Override Builder appendAllValuesToCurrentPosition(Block block); /** diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Block.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Block.java index c5d6780e84685..d7c3a5cb9bfab 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Block.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/Block.java @@ -12,6 +12,7 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; +import org.elasticsearch.index.mapper.BlockLoader; import java.util.List; @@ -160,7 +161,7 @@ static Block constantNullBlock(int positions, BlockFactory blockFactory) { * Builds {@link Block}s. Typically, you use one of it's direct supinterfaces like {@link IntBlock.Builder}. * This is {@link Releasable} and should be released after building the block or if building the block fails. */ - interface Builder extends Releasable { + interface Builder extends BlockLoader.Builder, Releasable { /** * Appends a null value to the block. diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/SingletonOrdinalsBuilder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/SingletonOrdinalsBuilder.java new file mode 100644 index 0000000000000..703d882b91029 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/SingletonOrdinalsBuilder.java @@ -0,0 +1,147 @@ +/* + * 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.apache.lucene.index.SortedDocValues; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.RamUsageEstimator; +import org.elasticsearch.compute.operator.BreakingBytesRefBuilder; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.index.mapper.BlockLoader; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; + +public class SingletonOrdinalsBuilder implements BlockLoader.SingletonOrdinalsBuilder, Releasable, Block.Builder { + private final BlockFactory blockFactory; + private final SortedDocValues docValues; + private int[] ords; + private int count; + + public SingletonOrdinalsBuilder(BlockFactory blockFactory, SortedDocValues docValues, int count) { + this.blockFactory = blockFactory; + this.docValues = docValues; + blockFactory.adjustBreaker(ordsSize(count), false); + this.ords = new int[count]; + } + + @Override + public SingletonOrdinalsBuilder appendNull() { + ords[count++] = -1; // real ords can't be < 0, so we use -1 as null + return this; + } + + @Override + public SingletonOrdinalsBuilder appendOrd(int value) { + ords[count++] = value; + return this; + } + + int[] ords() { + return ords; + } + + @Override + public SingletonOrdinalsBuilder beginPositionEntry() { + throw new UnsupportedOperationException("should only have one value per doc"); + } + + @Override + public SingletonOrdinalsBuilder endPositionEntry() { + throw new UnsupportedOperationException("should only have one value per doc"); + } + + @Override + public BytesRefBlock build() { + try { + long breakerSize = ordsSize(ords.length); + // Increment breaker for sorted ords. + blockFactory.adjustBreaker(breakerSize, false); + try { + int[] sortedOrds = ords.clone(); + Arrays.sort(sortedOrds); + int uniqueCount = compactToUnique(sortedOrds); + + try (BreakingBytesRefBuilder copies = new BreakingBytesRefBuilder(blockFactory.breaker(), "ords")) { + long offsetsAndLength = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + (uniqueCount + 1) * Integer.BYTES; + blockFactory.adjustBreaker(offsetsAndLength, false); + breakerSize += offsetsAndLength; + int[] offsets = new int[uniqueCount + 1]; + for (int o = 0; o < uniqueCount; o++) { + BytesRef v = docValues.lookupOrd(sortedOrds[o]); + offsets[o] = copies.length(); + copies.append(v); + } + offsets[uniqueCount] = copies.length(); + + /* + * It'd be better if BytesRefBlock could run off of a deduplicated list of + * blocks. It can't at the moment. So we copy many times. + */ + BytesRef scratch = new BytesRef(); + scratch.bytes = copies.bytes(); + try (BytesRefBlock.Builder builder = blockFactory.newBytesRefBlockBuilder(ords.length)) { + for (int i = 0; i < ords.length; i++) { + if (ords[i] == -1) { + builder.appendNull(); + continue; + } + int o = Arrays.binarySearch(sortedOrds, 0, uniqueCount, ords[i]); + assert 0 <= o && o < uniqueCount; + scratch.offset = offsets[o]; + scratch.length = offsets[o + 1] - scratch.offset; + builder.appendBytesRef(scratch); + } + return builder.build(); + } + } + } finally { + blockFactory.adjustBreaker(-breakerSize, false); + } + } catch (IOException e) { + throw new UncheckedIOException("error resolving ordinals", e); + } + } + + @Override + public void close() { + blockFactory.adjustBreaker(-ordsSize(ords.length), false); + } + + @Override + public Block.Builder appendAllValuesToCurrentPosition(Block block) { + throw new UnsupportedOperationException(); + } + + @Override + public Block.Builder copyFrom(Block block, int beginInclusive, int endExclusive) { + throw new UnsupportedOperationException(); + } + + @Override + public Block.Builder mvOrdering(Block.MvOrdering mvOrdering) { + throw new UnsupportedOperationException(); + } + + private static long ordsSize(int ordsCount) { + return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + ordsCount * Integer.BYTES; + } + + static int compactToUnique(int[] sortedOrds) { + Arrays.sort(sortedOrds); + int uniqueSize = 0; + int prev = -1; + for (int i = 0; i < sortedOrds.length; i++) { + if (sortedOrds[i] != prev) { + sortedOrds[uniqueSize++] = prev = sortedOrds[i]; + } + } + return uniqueSize; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st index 1dac4f1783e44..7f03e22332ccf 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/X-Block.java.st @@ -14,6 +14,7 @@ $endif$ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.mapper.BlockLoader; import java.io.IOException; @@ -222,11 +223,14 @@ $endif$ return blockFactory.newConstant$Type$BlockWith(value, positions); } - sealed interface Builder extends Block.Builder permits $Type$BlockBuilder { - + /** + * Builder for {@link $Type$Block} + */ + sealed interface Builder extends Block.Builder, BlockLoader.$Type$Builder permits $Type$BlockBuilder { /** * Appends a $type$ to the current entry. */ + @Override Builder append$Type$($type$ value); /** @@ -250,12 +254,11 @@ $endif$ @Override Builder mvOrdering(Block.MvOrdering mvOrdering); - // TODO boolean containsMvDups(); - /** * Appends the all values of the given block into a the current position * in this builder. */ + @Override Builder appendAllValuesToCurrentPosition(Block block); /** diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockDocValuesReader.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockDocValuesReader.java deleted file mode 100644 index 28a9359497393..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockDocValuesReader.java +++ /dev/null @@ -1,698 +0,0 @@ -/* - * 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.lucene; - -import org.apache.lucene.index.DocValues; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.NumericDocValues; -import org.apache.lucene.index.SortedNumericDocValues; -import org.elasticsearch.compute.data.Block; -import org.elasticsearch.compute.data.BooleanBlock; -import org.elasticsearch.compute.data.BytesRefBlock; -import org.elasticsearch.compute.data.DoubleBlock; -import org.elasticsearch.compute.data.ElementType; -import org.elasticsearch.compute.data.IntBlock; -import org.elasticsearch.compute.data.IntVector; -import org.elasticsearch.compute.data.LongBlock; -import org.elasticsearch.index.fielddata.FieldData; -import org.elasticsearch.index.fielddata.NumericDoubleValues; -import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceType; - -import java.io.IOException; - -/** - * A reader that supports reading doc-values from a Lucene segment in Block fashion. - */ -public abstract class BlockDocValuesReader { - - protected final Thread creationThread; - - public BlockDocValuesReader() { - this.creationThread = Thread.currentThread(); - } - - /** - * Returns the current doc that this reader is on. - */ - public abstract int docID(); - - /** - * The {@link Block.Builder} for data of this type. - */ - public abstract Block.Builder builder(int positionCount); - - /** - * Reads the values of the given documents specified in the input block - */ - public abstract Block readValues(IntVector docs) throws IOException; - - /** - * Reads the values of the given document into the builder - */ - public abstract void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException; - - /** - * Checks if the reader can be used to read a range documents starting with the given docID by the current thread. - */ - public static boolean canReuse(BlockDocValuesReader reader, int startingDocID) { - return reader != null && reader.creationThread == Thread.currentThread() && reader.docID() <= startingDocID; - } - - public static BlockDocValuesReader createBlockReader( - ValuesSource valuesSource, - ValuesSourceType valuesSourceType, - ElementType elementType, - LeafReaderContext leafReaderContext - ) throws IOException { - if (valuesSourceType instanceof UnsupportedValueSourceType) { - final UnsupportedValueSource bytesVS = (UnsupportedValueSource) valuesSource; - final SortedBinaryDocValues bytesValues = bytesVS.bytesValues(leafReaderContext); - return new BytesValuesReader(bytesValues); - } - if (CoreValuesSourceType.NUMERIC.equals(valuesSourceType) || CoreValuesSourceType.DATE.equals(valuesSourceType)) { - ValuesSource.Numeric numericVS = (ValuesSource.Numeric) valuesSource; - if (numericVS.isFloatingPoint()) { - if (elementType != ElementType.DOUBLE) { - throw new UnsupportedOperationException("can't extract [" + elementType + "] from floating point fields"); - } - final SortedNumericDoubleValues doubleValues = numericVS.doubleValues(leafReaderContext); - final NumericDoubleValues singleton = FieldData.unwrapSingleton(doubleValues); - if (singleton != null) { - return new DoubleSingletonValuesReader(singleton); - } - return new DoubleValuesReader(doubleValues); - } else { - final SortedNumericDocValues longValues = numericVS.longValues(leafReaderContext); - final NumericDocValues singleton = DocValues.unwrapSingleton(longValues); - if (singleton != null) { - return switch (elementType) { - case LONG -> new LongSingletonValuesReader(singleton); - case INT -> new IntSingletonValuesReader(singleton); - default -> throw new UnsupportedOperationException("can't extract [" + elementType + "] from integer fields"); - }; - } - return switch (elementType) { - case LONG -> new LongValuesReader(longValues); - case INT -> new IntValuesReader(longValues); - default -> throw new UnsupportedOperationException("can't extract [" + elementType + "] from integer fields"); - }; - } - } - if (CoreValuesSourceType.KEYWORD.equals(valuesSourceType) || CoreValuesSourceType.IP.equals(valuesSourceType)) { - if (elementType != ElementType.BYTES_REF) { - throw new UnsupportedOperationException("can't extract [" + elementType + "] from keywords"); - } - final ValuesSource.Bytes bytesVS = (ValuesSource.Bytes) valuesSource; - final SortedBinaryDocValues bytesValues = bytesVS.bytesValues(leafReaderContext); - return new BytesValuesReader(bytesValues); - } - if (CoreValuesSourceType.BOOLEAN.equals(valuesSourceType)) { - if (elementType != ElementType.BOOLEAN) { - throw new UnsupportedOperationException("can't extract [" + elementType + "] from booleans"); - } - ValuesSource.Numeric numericVS = (ValuesSource.Numeric) valuesSource; - final SortedNumericDocValues longValues = numericVS.longValues(leafReaderContext); - final NumericDocValues singleton = DocValues.unwrapSingleton(longValues); - if (singleton != null) { - return new BooleanSingletonValuesReader(singleton); - } - return new BooleanValuesReader(longValues); - } - if (valuesSourceType instanceof NullValueSourceType) { - return new NullValuesReader(); - } - throw new IllegalArgumentException("Field type [" + valuesSourceType.typeName() + "] is not supported"); - } - - private static class LongSingletonValuesReader extends BlockDocValuesReader { - private final NumericDocValues numericDocValues; - - LongSingletonValuesReader(NumericDocValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public LongBlock.Builder builder(int positionCount) { - return LongBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public LongBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - int lastDoc = -1; - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < lastDoc) { - throw new IllegalStateException("docs within same block must be in order"); - } - if (numericDocValues.advanceExact(doc)) { - blockBuilder.appendLong(numericDocValues.longValue()); - } else { - blockBuilder.appendNull(); - } - lastDoc = doc; - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - LongBlock.Builder blockBuilder = (LongBlock.Builder) builder; - if (numericDocValues.advanceExact(docId)) { - blockBuilder.appendLong(numericDocValues.longValue()); - } else { - blockBuilder.appendNull(); - } - } - - @Override - public int docID() { - return numericDocValues.docID(); - } - - @Override - public String toString() { - return "LongSingletonValuesReader"; - } - } - - private static class LongValuesReader extends BlockDocValuesReader { - private final SortedNumericDocValues numericDocValues; - private int docID = -1; - - LongValuesReader(SortedNumericDocValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public LongBlock.Builder builder(int positionCount) { - return LongBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public LongBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < this.docID) { - throw new IllegalStateException("docs within same block must be in order"); - } - read(doc, blockBuilder); - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - read(docId, (LongBlock.Builder) builder); - } - - private void read(int doc, LongBlock.Builder builder) throws IOException { - this.docID = doc; - if (false == numericDocValues.advanceExact(doc)) { - builder.appendNull(); - return; - } - int count = numericDocValues.docValueCount(); - if (count == 1) { - builder.appendLong(numericDocValues.nextValue()); - return; - } - builder.beginPositionEntry(); - for (int v = 0; v < count; v++) { - builder.appendLong(numericDocValues.nextValue()); - } - builder.endPositionEntry(); - } - - @Override - public int docID() { - // There is a .docID on the numericDocValues but it is often not implemented. - return docID; - } - - @Override - public String toString() { - return "LongValuesReader"; - } - } - - private static class IntSingletonValuesReader extends BlockDocValuesReader { - private final NumericDocValues numericDocValues; - - IntSingletonValuesReader(NumericDocValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public IntBlock.Builder builder(int positionCount) { - return IntBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public IntBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - int lastDoc = -1; - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < lastDoc) { - throw new IllegalStateException("docs within same block must be in order"); - } - if (numericDocValues.advanceExact(doc)) { - blockBuilder.appendInt(Math.toIntExact(numericDocValues.longValue())); - } else { - blockBuilder.appendNull(); - } - lastDoc = doc; - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - IntBlock.Builder blockBuilder = (IntBlock.Builder) builder; - if (numericDocValues.advanceExact(docId)) { - blockBuilder.appendInt(Math.toIntExact(numericDocValues.longValue())); - } else { - blockBuilder.appendNull(); - } - } - - @Override - public int docID() { - return numericDocValues.docID(); - } - - @Override - public String toString() { - return "LongSingletonValuesReader"; - } - } - - private static class IntValuesReader extends BlockDocValuesReader { - private final SortedNumericDocValues numericDocValues; - private int docID = -1; - - IntValuesReader(SortedNumericDocValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public IntBlock.Builder builder(int positionCount) { - return IntBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public IntBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < this.docID) { - // TODO this may not be true after sorting many docs in a single segment. - throw new IllegalStateException("docs within same block must be in order"); - } - read(doc, blockBuilder); - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - read(docId, (IntBlock.Builder) builder); - } - - private void read(int doc, IntBlock.Builder builder) throws IOException { - this.docID = doc; - if (false == numericDocValues.advanceExact(doc)) { - builder.appendNull(); - return; - } - int count = numericDocValues.docValueCount(); - if (count == 1) { - builder.appendInt(Math.toIntExact(numericDocValues.nextValue())); - return; - } - builder.beginPositionEntry(); - for (int v = 0; v < count; v++) { - builder.appendInt(Math.toIntExact(numericDocValues.nextValue())); - } - builder.endPositionEntry(); - } - - @Override - public int docID() { - // There is a .docID on on the numericDocValues but it is often not implemented. - return docID; - } - - @Override - public String toString() { - return "LongValuesReader"; - } - } - - private static class DoubleSingletonValuesReader extends BlockDocValuesReader { - private final NumericDoubleValues numericDocValues; - private int docID = -1; - - DoubleSingletonValuesReader(NumericDoubleValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public DoubleBlock.Builder builder(int positionCount) { - return DoubleBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public DoubleBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - int lastDoc = -1; - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < lastDoc) { - throw new IllegalStateException("docs within same block must be in order"); - } - if (numericDocValues.advanceExact(doc)) { - blockBuilder.appendDouble(numericDocValues.doubleValue()); - } else { - blockBuilder.appendNull(); - } - lastDoc = doc; - this.docID = doc; - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - this.docID = docId; - DoubleBlock.Builder blockBuilder = (DoubleBlock.Builder) builder; - if (numericDocValues.advanceExact(this.docID)) { - blockBuilder.appendDouble(numericDocValues.doubleValue()); - } else { - blockBuilder.appendNull(); - } - } - - @Override - public int docID() { - return docID; - } - - @Override - public String toString() { - return "DoubleSingletonValuesReader"; - } - } - - private static class DoubleValuesReader extends BlockDocValuesReader { - private final SortedNumericDoubleValues numericDocValues; - private int docID = -1; - - DoubleValuesReader(SortedNumericDoubleValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public DoubleBlock.Builder builder(int positionCount) { - return DoubleBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public DoubleBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < this.docID) { - throw new IllegalStateException("docs within same block must be in order"); - } - read(doc, blockBuilder); - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - read(docId, (DoubleBlock.Builder) builder); - } - - private void read(int doc, DoubleBlock.Builder builder) throws IOException { - this.docID = doc; - if (false == numericDocValues.advanceExact(doc)) { - builder.appendNull(); - return; - } - int count = numericDocValues.docValueCount(); - if (count == 1) { - builder.appendDouble(numericDocValues.nextValue()); - return; - } - builder.beginPositionEntry(); - for (int v = 0; v < count; v++) { - builder.appendDouble(numericDocValues.nextValue()); - } - builder.endPositionEntry(); - } - - @Override - public int docID() { - return docID; - } - - @Override - public String toString() { - return "DoubleValuesReader"; - } - } - - private static class BytesValuesReader extends BlockDocValuesReader { - private final SortedBinaryDocValues binaryDV; - private int docID = -1; - - BytesValuesReader(SortedBinaryDocValues binaryDV) { - this.binaryDV = binaryDV; - } - - @Override - public BytesRefBlock.Builder builder(int positionCount) { - return BytesRefBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public BytesRefBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - for (int i = 0; i < docs.getPositionCount(); i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < this.docID) { - throw new IllegalStateException("docs within same block must be in order"); - } - read(doc, blockBuilder); - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - read(docId, (BytesRefBlock.Builder) builder); - } - - private void read(int doc, BytesRefBlock.Builder builder) throws IOException { - this.docID = doc; - if (false == binaryDV.advanceExact(doc)) { - builder.appendNull(); - return; - } - int count = binaryDV.docValueCount(); - if (count == 1) { - builder.appendBytesRef(binaryDV.nextValue()); - return; - } - builder.beginPositionEntry(); - for (int v = 0; v < count; v++) { - builder.appendBytesRef(binaryDV.nextValue()); - } - builder.endPositionEntry(); - } - - @Override - public int docID() { - return docID; - } - - @Override - public String toString() { - return "BytesValuesReader"; - } - } - - private static class BooleanSingletonValuesReader extends BlockDocValuesReader { - private final NumericDocValues numericDocValues; - - BooleanSingletonValuesReader(NumericDocValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public BooleanBlock.Builder builder(int positionCount) { - return BooleanBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public BooleanBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - int lastDoc = -1; - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < lastDoc) { - throw new IllegalStateException("docs within same block must be in order"); - } - if (numericDocValues.advanceExact(doc)) { - blockBuilder.appendBoolean(numericDocValues.longValue() != 0); - } else { - blockBuilder.appendNull(); - } - lastDoc = doc; - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - BooleanBlock.Builder blockBuilder = (BooleanBlock.Builder) builder; - if (numericDocValues.advanceExact(docId)) { - blockBuilder.appendBoolean(numericDocValues.longValue() != 0); - } else { - blockBuilder.appendNull(); - } - } - - @Override - public int docID() { - return numericDocValues.docID(); - } - - @Override - public String toString() { - return getClass().getSimpleName(); - } - } - - private static class BooleanValuesReader extends BlockDocValuesReader { - private final SortedNumericDocValues numericDocValues; - private int docID = -1; - - BooleanValuesReader(SortedNumericDocValues numericDocValues) { - this.numericDocValues = numericDocValues; - } - - @Override - public BooleanBlock.Builder builder(int positionCount) { - return BooleanBlock.newBlockBuilder(positionCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); - } - - @Override - public BooleanBlock readValues(IntVector docs) throws IOException { - final int positionCount = docs.getPositionCount(); - var blockBuilder = builder(positionCount); - for (int i = 0; i < positionCount; i++) { - int doc = docs.getInt(i); - // docs within same block must be in order - if (doc < this.docID) { - throw new IllegalStateException("docs within same block must be in order"); - } - read(doc, blockBuilder); - } - return blockBuilder.build(); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) throws IOException { - read(docId, (BooleanBlock.Builder) builder); - } - - private void read(int doc, BooleanBlock.Builder builder) throws IOException { - this.docID = doc; - if (false == numericDocValues.advanceExact(doc)) { - builder.appendNull(); - return; - } - int count = numericDocValues.docValueCount(); - if (count == 1) { - builder.appendBoolean(numericDocValues.nextValue() != 0); - return; - } - builder.beginPositionEntry(); - for (int v = 0; v < count; v++) { - builder.appendBoolean(numericDocValues.nextValue() != 0); - } - builder.endPositionEntry(); - } - - @Override - public int docID() { - // There is a .docID on the numericDocValues but it is often not implemented. - return docID; - } - - @Override - public String toString() { - return getClass().getSimpleName(); - } - } - - private static class NullValuesReader extends BlockDocValuesReader { - private int docID = -1; - - @Override - public Block.Builder builder(int positionCount) { - return ElementType.NULL.newBlockBuilder(positionCount); - } - - @Override - public Block readValues(IntVector docs) throws IOException { - return Block.constantNullBlock(docs.getPositionCount()); - } - - @Override - public void readValuesFromSingleDoc(int docId, Block.Builder builder) { - this.docID = docId; - builder.appendNull(); - } - - @Override - public int docID() { - return docID; - } - - @Override - public String toString() { - return getClass().getSimpleName(); - } - } -} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockReaderFactories.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockReaderFactories.java new file mode 100644 index 0000000000000..a0d08bc798fbb --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/BlockReaderFactories.java @@ -0,0 +1,108 @@ +/* + * 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.lucene; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.SortedSetDocValues; +import org.elasticsearch.common.logging.HeaderWarning; +import org.elasticsearch.index.mapper.BlockDocValuesReader; +import org.elasticsearch.index.mapper.BlockLoader; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.lookup.SearchLookup; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +/** + * Resolves *how* ESQL loads field values. + */ +public final class BlockReaderFactories { + private BlockReaderFactories() {} + + /** + * Resolves *how* ESQL loads field values. + * @param searchContexts a search context per search index we're loading + * field from + * @param fieldName the name of the field to load + * @param asUnsupportedSource should the field be loaded as "unsupported"? + * These will always have {@code null} values + */ + public static List factories( + List searchContexts, + String fieldName, + boolean asUnsupportedSource + ) { + List factories = new ArrayList<>(searchContexts.size()); + + for (SearchContext searchContext : searchContexts) { + SearchExecutionContext ctx = searchContext.getSearchExecutionContext(); + if (asUnsupportedSource) { + factories.add(loaderToFactory(ctx.getIndexReader(), BlockDocValuesReader.nulls())); + continue; + } + MappedFieldType fieldType = ctx.getFieldType(fieldName); + if (fieldType == null) { + // the field does not exist in this context + factories.add(loaderToFactory(ctx.getIndexReader(), BlockDocValuesReader.nulls())); + continue; + } + BlockLoader loader = fieldType.blockLoader(new MappedFieldType.BlockLoaderContext() { + @Override + public String indexName() { + return ctx.getFullyQualifiedIndex().getName(); + } + + @Override + public SearchLookup lookup() { + return ctx.lookup(); + } + + @Override + public Set sourcePaths(String name) { + return ctx.sourcePath(name); + } + }); + if (loader == null) { + HeaderWarning.addWarning("Field [{}] cannot be retrieved, it is unsupported or not indexed; returning null", fieldName); + factories.add(loaderToFactory(ctx.getIndexReader(), BlockDocValuesReader.nulls())); + continue; + } + factories.add(loaderToFactory(ctx.getIndexReader(), loader)); + } + + return factories; + } + + /** + * Converts a {@link BlockLoader}, something defined in core elasticsearch at + * the field level, into a {@link BlockDocValuesReader.Factory} which can be + * used inside ESQL. + */ + public static BlockDocValuesReader.Factory loaderToFactory(IndexReader reader, BlockLoader loader) { + return new BlockDocValuesReader.Factory() { + @Override + public BlockDocValuesReader build(int segment) throws IOException { + return loader.reader(reader.leaves().get(segment)); + } + + @Override + public boolean supportsOrdinals() { + return loader.supportsOrdinals(); + } + + @Override + public SortedSetDocValues ordinals(int segment) throws IOException { + return loader.ordinals(reader.leaves().get(segment)); + } + }; + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSource.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSource.java deleted file mode 100644 index fc9807b2e2410..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSource.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.lucene; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.Rounding; -import org.elasticsearch.index.fielddata.DocValueBits; -import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.search.aggregations.support.ValuesSource; - -import java.io.IOException; -import java.util.function.Function; - -public class NullValueSource extends ValuesSource { - - @Override - public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { - - return new SortedBinaryDocValues() { - @Override - public boolean advanceExact(int doc) throws IOException { - return true; - } - - @Override - public int docValueCount() { - return 1; - } - - @Override - public BytesRef nextValue() throws IOException { - return null; - } - }; - } - - @Override - public DocValueBits docsWithValue(LeafReaderContext context) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - protected Function roundingPreparer(AggregationContext context) throws IOException { - throw new UnsupportedOperationException(); - } -} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSourceType.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSourceType.java deleted file mode 100644 index fd354bd9e1a0b..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/NullValueSourceType.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.lucene; - -import org.elasticsearch.script.AggregationScript; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.search.aggregations.support.FieldContext; -import org.elasticsearch.search.aggregations.support.ValueType; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceType; - -public class NullValueSourceType implements ValuesSourceType { - - @Override - public ValuesSource getEmpty() { - throw new UnsupportedOperationException(); - } - - @Override - public ValuesSource getScript(AggregationScript.LeafFactory script, ValueType scriptValueType) { - throw new UnsupportedOperationException(); - } - - @Override - public ValuesSource getField(FieldContext fieldContext, AggregationScript.LeafFactory script) { - throw new UnsupportedOperationException(); - } - - @Override - public ValuesSource replaceMissing( - ValuesSource valuesSource, - Object rawMissing, - DocValueFormat docValueFormat, - AggregationContext context - ) { - throw new UnsupportedOperationException(); - } - - @Override - public String typeName() { - return null; - } - -} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSourceInfo.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSourceInfo.java deleted file mode 100644 index e4dffdfe72c4d..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSourceInfo.java +++ /dev/null @@ -1,15 +0,0 @@ -/* - * 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.lucene; - -import org.apache.lucene.index.IndexReader; -import org.elasticsearch.compute.data.ElementType; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceType; - -public record ValueSourceInfo(ValuesSourceType type, ValuesSource source, ElementType elementType, IndexReader reader) {} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSources.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSources.java deleted file mode 100644 index 29a539b1e068e..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValueSources.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * 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.lucene; - -import org.apache.lucene.index.DocValues; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.logging.HeaderWarning; -import org.elasticsearch.compute.data.ElementType; -import org.elasticsearch.index.fielddata.FieldDataContext; -import org.elasticsearch.index.fielddata.IndexFieldData; -import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.index.fielddata.SourceValueFetcherSortedBinaryIndexFieldData; -import org.elasticsearch.index.fielddata.StoredFieldSortedBinaryIndexFieldData; -import org.elasticsearch.index.mapper.IdFieldMapper; -import org.elasticsearch.index.mapper.KeywordFieldMapper; -import org.elasticsearch.index.mapper.MappedFieldType; -import org.elasticsearch.index.mapper.SourceValueFetcher; -import org.elasticsearch.index.mapper.TextFieldMapper; -import org.elasticsearch.index.query.SearchExecutionContext; -import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; -import org.elasticsearch.search.aggregations.support.FieldContext; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.internal.ShardSearchRequest; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -public final class ValueSources { - - public static final String MATCH_ONLY_TEXT = "match_only_text"; - - private ValueSources() {} - - public static List sources( - List searchContexts, - String fieldName, - boolean asUnsupportedSource, - ElementType elementType - ) { - List sources = new ArrayList<>(searchContexts.size()); - - for (SearchContext searchContext : searchContexts) { - // TODO: remove this workaround - // Create a separate SearchExecutionContext for each ValuesReader, as it seems that - // the synthetic source doesn't work properly with inter-segment or intra-segment parallelism. - ShardSearchRequest shardRequest = searchContext.request(); - SearchExecutionContext ctx = searchContext.readerContext() - .indexService() - .newSearchExecutionContext( - shardRequest.shardId().id(), - shardRequest.shardRequestIndex(), - searchContext.searcher(), - shardRequest::nowInMillis, - shardRequest.getClusterAlias(), - shardRequest.getRuntimeMappings() - ); - var fieldType = ctx.getFieldType(fieldName); - if (fieldType == null) { - sources.add(new ValueSourceInfo(new NullValueSourceType(), new NullValueSource(), elementType, ctx.getIndexReader())); - continue; // the field does not exist in this context - } - if (asUnsupportedSource) { - sources.add( - new ValueSourceInfo( - new UnsupportedValueSourceType(fieldType.typeName()), - new UnsupportedValueSource(null), - elementType, - ctx.getIndexReader() - ) - ); - HeaderWarning.addWarning("Field [{}] cannot be retrieved, it is unsupported or not indexed; returning null", fieldName); - continue; - } - - if (fieldType.hasDocValues() == false) { - // MatchOnlyTextFieldMapper class lives in the mapper-extras module. We use string equality - // for the field type name to avoid adding a dependency to the module - if (fieldType instanceof KeywordFieldMapper.KeywordFieldType - || fieldType instanceof TextFieldMapper.TextFieldType tft && (tft.isSyntheticSource() == false || tft.isStored()) - || MATCH_ONLY_TEXT.equals(fieldType.typeName())) { - ValuesSource vs = textValueSource(ctx, fieldType); - sources.add(new ValueSourceInfo(CoreValuesSourceType.KEYWORD, vs, elementType, ctx.getIndexReader())); - continue; - } - - if (IdFieldMapper.NAME.equals(fieldType.name())) { - ValuesSource vs = new IdValueSource(new IdFieldIndexFieldData(CoreValuesSourceType.KEYWORD)); - sources.add(new ValueSourceInfo(CoreValuesSourceType.KEYWORD, vs, elementType, ctx.getIndexReader())); - continue; - } - } - - IndexFieldData fieldData; - try { - fieldData = ctx.getForField(fieldType, MappedFieldType.FielddataOperation.SEARCH); - } catch (IllegalArgumentException e) { - sources.add(unsupportedValueSource(elementType, ctx, fieldType, e)); - HeaderWarning.addWarning("Field [{}] cannot be retrieved, it is unsupported or not indexed; returning null", fieldName); - continue; - } - var fieldContext = new FieldContext(fieldName, fieldData, fieldType); - var vsType = fieldData.getValuesSourceType(); - var vs = vsType.getField(fieldContext, null); - sources.add(new ValueSourceInfo(vsType, vs, elementType, ctx.getIndexReader())); - } - - return sources; - } - - private static ValueSourceInfo unsupportedValueSource( - ElementType elementType, - SearchExecutionContext ctx, - MappedFieldType fieldType, - IllegalArgumentException e - ) { - return switch (elementType) { - case BYTES_REF -> new ValueSourceInfo( - new UnsupportedValueSourceType(fieldType.typeName()), - new UnsupportedValueSource(null), - elementType, - ctx.getIndexReader() - ); - case LONG, INT -> new ValueSourceInfo( - CoreValuesSourceType.NUMERIC, - ValuesSource.Numeric.EMPTY, - elementType, - ctx.getIndexReader() - ); - case BOOLEAN -> new ValueSourceInfo( - CoreValuesSourceType.BOOLEAN, - ValuesSource.Numeric.EMPTY, - elementType, - ctx.getIndexReader() - ); - case DOUBLE -> new ValueSourceInfo(CoreValuesSourceType.NUMERIC, new ValuesSource.Numeric() { - @Override - public boolean isFloatingPoint() { - return true; - } - - @Override - public SortedNumericDocValues longValues(LeafReaderContext context) { - return DocValues.emptySortedNumeric(); - } - - @Override - public SortedNumericDoubleValues doubleValues(LeafReaderContext context) throws IOException { - return org.elasticsearch.index.fielddata.FieldData.emptySortedNumericDoubles(); - } - - @Override - public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { - return org.elasticsearch.index.fielddata.FieldData.emptySortedBinary(); - } - }, elementType, ctx.getIndexReader()); - default -> throw e; - }; - } - - private static TextValueSource textValueSource(SearchExecutionContext ctx, MappedFieldType fieldType) { - if (fieldType.isStored()) { - IndexFieldData fieldData = new StoredFieldSortedBinaryIndexFieldData( - fieldType.name(), - CoreValuesSourceType.KEYWORD, - TextValueSource.TextDocValuesFieldWrapper::new - ) { - @Override - protected BytesRef storedToBytesRef(Object stored) { - return new BytesRef((String) stored); - } - }; - return new TextValueSource(fieldData); - } - - FieldDataContext fieldDataContext = new FieldDataContext( - ctx.getFullyQualifiedIndex().getName(), - () -> ctx.lookup().forkAndTrackFieldReferences(fieldType.name()), - ctx::sourcePath, - MappedFieldType.FielddataOperation.SEARCH - ); - IndexFieldData fieldData = new SourceValueFetcherSortedBinaryIndexFieldData.Builder( - fieldType.name(), - CoreValuesSourceType.KEYWORD, - SourceValueFetcher.toString(fieldDataContext.sourcePathsLookup().apply(fieldType.name())), - fieldDataContext.lookupSupplier().get(), - TextValueSource.TextDocValuesFieldWrapper::new - ).build(null, null); // Neither cache nor breakerService are used by SourceValueFetcherSortedBinaryIndexFieldData builder - return new TextValueSource(fieldData); - } -} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java index 83fc902bd5077..8b1c4f78825ad 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java @@ -7,18 +7,24 @@ package org.elasticsearch.compute.lucene; -import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedDocValues; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.DocBlock; import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.data.SingletonOrdinalsBuilder; import org.elasticsearch.compute.operator.AbstractPageMappingOperator; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.index.mapper.BlockDocValuesReader; +import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.xcontent.XContentBuilder; @@ -28,7 +34,6 @@ import java.util.Map; import java.util.Objects; import java.util.TreeMap; -import java.util.function.Supplier; /** * Operator that extracts doc_values from a Lucene index out of pages that have been produced by {@link LuceneSourceOperator} @@ -43,12 +48,12 @@ public class ValuesSourceReaderOperator extends AbstractPageMappingOperator { * @param docChannel the channel containing the shard, leaf/segment and doc id * @param field the lucene field being loaded */ - public record ValuesSourceReaderOperatorFactory(Supplier> sources, int docChannel, String field) + public record ValuesSourceReaderOperatorFactory(List sources, int docChannel, String field) implements OperatorFactory { @Override public Operator get(DriverContext driverContext) { - return new ValuesSourceReaderOperator(sources.get(), docChannel, field); + return new ValuesSourceReaderOperator(sources, docChannel, field); } @Override @@ -57,9 +62,14 @@ public String describe() { } } - private final List sources; + /** + * A list, one entry per shard, of factories for {@link BlockDocValuesReader}s + * which perform the actual reading. + */ + private final List factories; private final int docChannel; private final String field; + private final ComputeBlockLoaderFactory blockFactory; private BlockDocValuesReader lastReader; private int lastShard = -1; @@ -69,14 +79,15 @@ public String describe() { /** * Creates a new extractor - * @param sources the value source, type and index readers to use for extraction + * @param factories builds {@link BlockDocValuesReader} * @param docChannel the channel containing the shard, leaf/segment and doc id * @param field the lucene field being loaded */ - public ValuesSourceReaderOperator(List sources, int docChannel, String field) { - this.sources = sources; + public ValuesSourceReaderOperator(List factories, int docChannel, String field) { + this.factories = factories; this.docChannel = docChannel; this.field = field; + this.blockFactory = new ComputeBlockLoaderFactory(BlockFactory.getNonBreakingInstance()); // TODO breaking! } @Override @@ -95,14 +106,26 @@ protected Page process(Page page) { private Block loadFromSingleLeaf(DocVector docVector) throws IOException { setupReader(docVector.shards().getInt(0), docVector.segments().getInt(0), docVector.docs().getInt(0)); - return lastReader.readValues(docVector.docs()); + return ((Block.Builder) lastReader.readValues(blockFactory, new BlockLoader.Docs() { + private final IntVector docs = docVector.docs(); + + @Override + public int count() { + return docs.getPositionCount(); + } + + @Override + public int get(int i) { + return docs.getInt(i); + } + })).build(); } private Block loadFromManyLeaves(DocVector docVector) throws IOException { int[] forwards = docVector.shardSegmentDocMapForwards(); int doc = docVector.docs().getInt(forwards[0]); setupReader(docVector.shards().getInt(forwards[0]), docVector.segments().getInt(forwards[0]), doc); - Block.Builder builder = lastReader.builder(forwards.length); + BlockLoader.Builder builder = lastReader.builder(blockFactory, forwards.length); lastReader.readValuesFromSingleDoc(doc, builder); for (int i = 1; i < forwards.length; i++) { int shard = docVector.shards().getInt(forwards[i]); @@ -114,16 +137,15 @@ private Block loadFromManyLeaves(DocVector docVector) throws IOException { lastReader.readValuesFromSingleDoc(doc, builder); } // TODO maybe it's better for downstream consumers if we perform a copy here. - return builder.build().filter(docVector.shardSegmentDocMapBackwards()); + return ((Block.Builder) builder).build().filter(docVector.shardSegmentDocMapBackwards()); } private void setupReader(int shard, int segment, int doc) throws IOException { if (lastSegment == segment && lastShard == shard && BlockDocValuesReader.canReuse(lastReader, doc)) { return; } - var info = sources.get(shard); - LeafReaderContext leafReaderContext = info.reader().leaves().get(segment); - lastReader = BlockDocValuesReader.createBlockReader(info.source(), info.type(), info.elementType(), leafReaderContext); + + lastReader = factories.get(shard).build(segment); lastShard = shard; lastSegment = segment; readersBuilt.compute(lastReader.toString(), (k, v) -> v == null ? 1 : v + 1); @@ -203,4 +225,72 @@ public String toString() { return Strings.toString(this); } } + + private static class ComputeBlockLoaderFactory implements BlockLoader.BuilderFactory { + private final BlockFactory factory; + + private ComputeBlockLoaderFactory(BlockFactory factory) { + this.factory = factory; + } + + @Override + public BlockLoader.BooleanBuilder booleansFromDocValues(int expectedCount) { + return factory.newBooleanBlockBuilder(expectedCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); + } + + @Override + public BlockLoader.BooleanBuilder booleans(int expectedCount) { + return factory.newBooleanBlockBuilder(expectedCount); + } + + @Override + public BlockLoader.BytesRefBuilder bytesRefsFromDocValues(int expectedCount) { + return factory.newBytesRefBlockBuilder(expectedCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); + } + + @Override + public BlockLoader.BytesRefBuilder bytesRefs(int expectedCount) { + return factory.newBytesRefBlockBuilder(expectedCount); + } + + @Override + public BlockLoader.DoubleBuilder doublesFromDocValues(int expectedCount) { + return factory.newDoubleBlockBuilder(expectedCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); + } + + @Override + public BlockLoader.DoubleBuilder doubles(int expectedCount) { + return factory.newDoubleBlockBuilder(expectedCount); + } + + @Override + public BlockLoader.IntBuilder intsFromDocValues(int expectedCount) { + return factory.newIntBlockBuilder(expectedCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); + } + + @Override + public BlockLoader.IntBuilder ints(int expectedCount) { + return factory.newIntBlockBuilder(expectedCount); + } + + @Override + public BlockLoader.LongBuilder longsFromDocValues(int expectedCount) { + return factory.newLongBlockBuilder(expectedCount).mvOrdering(Block.MvOrdering.DEDUPLICATED_AND_SORTED_ASCENDING); + } + + @Override + public BlockLoader.LongBuilder longs(int expectedCount) { + return factory.newLongBlockBuilder(expectedCount); + } + + @Override + public BlockLoader.Builder nulls(int expectedCount) { + return ElementType.NULL.newBlockBuilder(expectedCount, factory); + } + + @Override + public BlockLoader.SingletonOrdinalsBuilder singletonOrdinalsBuilder(SortedDocValues ordinals, int count) { + return new SingletonOrdinalsBuilder(factory, ordinals, count); + } + } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java index 216627f996cad..2c5eedb6d8bbe 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java @@ -7,11 +7,11 @@ package org.elasticsearch.compute.operator; -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.CheckedSupplier; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.compute.Describable; @@ -24,15 +24,15 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.DocBlock; import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; -import org.elasticsearch.compute.lucene.ValueSourceInfo; import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; import org.elasticsearch.compute.operator.HashAggregationOperator.GroupSpec; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; -import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.index.mapper.BlockDocValuesReader; import java.io.IOException; import java.io.UncheckedIOException; @@ -52,7 +52,8 @@ */ public class OrdinalsGroupingOperator implements Operator { public record OrdinalsGroupingOperatorFactory( - Supplier> sources, + List readerFactories, + ElementType groupingElementType, int docChannel, String groupingField, List aggregators, @@ -63,7 +64,8 @@ public record OrdinalsGroupingOperatorFactory( @Override public Operator get(DriverContext driverContext) { return new OrdinalsGroupingOperator( - sources.get(), + readerFactories, + groupingElementType, docChannel, groupingField, aggregators, @@ -79,11 +81,12 @@ public String describe() { } } - private final List sources; + private final List readerFactories; private final int docChannel; private final String groupingField; private final List aggregatorFactories; + private final ElementType groupingElementType; private final Map ordinalAggregators; private final BigArrays bigArrays; @@ -96,7 +99,8 @@ public String describe() { private ValuesAggregator valuesAggregator; public OrdinalsGroupingOperator( - List sources, + List readerFactories, + ElementType groupingElementType, int docChannel, String groupingField, List aggregatorFactories, @@ -105,7 +109,8 @@ public OrdinalsGroupingOperator( DriverContext driverContext ) { Objects.requireNonNull(aggregatorFactories); - this.sources = sources; + this.readerFactories = readerFactories; + this.groupingElementType = groupingElementType; this.docChannel = docChannel; this.groupingField = groupingField; this.aggregatorFactories = aggregatorFactories; @@ -126,22 +131,20 @@ public void addInput(Page page) { requireNonNull(page, "page is null"); DocVector docVector = page.getBlock(docChannel).asVector(); final int shardIndex = docVector.shards().getInt(0); - final var source = sources.get(shardIndex); + final var readerFactory = readerFactories.get(shardIndex); boolean pagePassed = false; try { - if (docVector.singleSegmentNonDecreasing() && source.source() instanceof ValuesSource.Bytes.WithOrdinals withOrdinals) { + if (docVector.singleSegmentNonDecreasing() && readerFactory.supportsOrdinals()) { final IntVector segmentIndexVector = docVector.segments(); assert segmentIndexVector.isConstant(); final OrdinalSegmentAggregator ordinalAggregator = this.ordinalAggregators.computeIfAbsent( new SegmentID(shardIndex, segmentIndexVector.getInt(0)), k -> { try { - final LeafReaderContext leafReaderContext = source.reader().leaves().get(k.segmentIndex); return new OrdinalSegmentAggregator( driverContext.blockFactory(), this::createGroupingAggregators, - withOrdinals, - leafReaderContext, + () -> readerFactory.ordinals(k.segmentIndex), bigArrays ); } catch (IOException e) { @@ -155,7 +158,8 @@ public void addInput(Page page) { if (valuesAggregator == null) { int channelIndex = page.getBlockCount(); // extractor will append a new block at the end valuesAggregator = new ValuesAggregator( - sources, + readerFactories, + groupingElementType, docChannel, groupingField, channelIndex, @@ -327,29 +331,26 @@ record SegmentID(int shardIndex, int segmentIndex) { static final class OrdinalSegmentAggregator implements Releasable, SeenGroupIds { private final BlockFactory blockFactory; private final List aggregators; - private final ValuesSource.Bytes.WithOrdinals withOrdinals; - private final LeafReaderContext leafReaderContext; + private final CheckedSupplier docValuesSupplier; private final BitArray visitedOrds; private BlockOrdinalsReader currentReader; OrdinalSegmentAggregator( BlockFactory blockFactory, Supplier> aggregatorsSupplier, - ValuesSource.Bytes.WithOrdinals withOrdinals, - LeafReaderContext leafReaderContext, + CheckedSupplier docValuesSupplier, BigArrays bigArrays ) throws IOException { boolean success = false; List groupingAggregators = null; BitArray bitArray = null; try { - final SortedSetDocValues sortedSetDocValues = withOrdinals.ordinalsValues(leafReaderContext); + final SortedSetDocValues sortedSetDocValues = docValuesSupplier.get(); bitArray = new BitArray(sortedSetDocValues.getValueCount(), bigArrays); groupingAggregators = aggregatorsSupplier.get(); this.currentReader = new BlockOrdinalsReader(sortedSetDocValues, blockFactory); this.blockFactory = blockFactory; - this.withOrdinals = withOrdinals; - this.leafReaderContext = leafReaderContext; + this.docValuesSupplier = docValuesSupplier; this.aggregators = groupingAggregators; this.visitedOrds = bitArray; success = true; @@ -369,7 +370,7 @@ void addInput(IntVector docs, Page page) { } if (BlockOrdinalsReader.canReuse(currentReader, docs.getInt(0)) == false) { - currentReader = new BlockOrdinalsReader(withOrdinals.ordinalsValues(leafReaderContext), blockFactory); + currentReader = new BlockOrdinalsReader(docValuesSupplier.get(), blockFactory); } try (IntBlock ordinals = currentReader.readOrdinalsAdded1(docs)) { for (int p = 0; p < ordinals.getPositionCount(); p++) { @@ -392,7 +393,7 @@ void addInput(IntVector docs, Page page) { } AggregatedResultIterator getResultIterator() throws IOException { - return new AggregatedResultIterator(aggregators, visitedOrds, withOrdinals.ordinalsValues(leafReaderContext)); + return new AggregatedResultIterator(aggregators, visitedOrds, docValuesSupplier.get()); } boolean seenNulls() { @@ -457,7 +458,8 @@ private static class ValuesAggregator implements Releasable { private final HashAggregationOperator aggregator; ValuesAggregator( - List sources, + List factories, + ElementType groupingElementType, int docChannel, String groupingField, int channelIndex, @@ -465,15 +467,10 @@ private static class ValuesAggregator implements Releasable { int maxPageSize, DriverContext driverContext ) { - this.extractor = new ValuesSourceReaderOperator(sources, docChannel, groupingField); + this.extractor = new ValuesSourceReaderOperator(factories, docChannel, groupingField); this.aggregator = new HashAggregationOperator( aggregatorFactories, - () -> BlockHash.build( - List.of(new GroupSpec(channelIndex, sources.get(0).elementType())), - driverContext, - maxPageSize, - false - ), + () -> BlockHash.build(List.of(new GroupSpec(channelIndex, groupingElementType)), driverContext, maxPageSize, false), driverContext ); } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java index 0b2cb20ecdabd..979777d4cfd03 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java @@ -154,7 +154,7 @@ void fetchPage() { final LoopControl loopControl = new LoopControl(); while (loopControl.isRunning()) { loopControl.exiting(); - // finish other sinks if one of them failed or sources no longer need pages. + // finish other sinks if one of them failed or source no longer need pages. boolean toFinishSinks = buffer.noMoreInputs() || failure.get() != null; remoteSink.fetchPageAsync(toFinishSinks, ActionListener.wrap(resp -> { Page page = resp.takePage(); @@ -249,7 +249,7 @@ protected void closeInternal() { /** * Add a listener, which will be notified when this exchange source handler is completed. An exchange source - * handler is consider completed when all exchange sources and sinks are completed and de-attached. + * handler is consider completed when all exchange factories and sinks are completed and de-attached. */ public void addCompletionListener(ActionListener listener) { completionFuture.addListener(listener); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java index 7b29f82f085c5..6951cdf4d56ca 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java @@ -14,7 +14,6 @@ import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.Term; import org.apache.lucene.search.Collector; import org.apache.lucene.search.IndexSearcher; @@ -46,10 +45,10 @@ import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.BlockReaderFactories; import org.elasticsearch.compute.lucene.DataPartitioning; import org.elasticsearch.compute.lucene.LuceneOperator; import org.elasticsearch.compute.lucene.LuceneSourceOperator; -import org.elasticsearch.compute.lucene.ValueSourceInfo; import org.elasticsearch.compute.operator.AbstractPageMappingOperator; import org.elasticsearch.compute.operator.Driver; import org.elasticsearch.compute.operator.DriverContext; @@ -62,13 +61,10 @@ import org.elasticsearch.compute.operator.SequenceLongBlockSourceOperator; import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.Releasables; -import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MapperServiceTestCase; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; -import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ESTestCase; @@ -80,7 +76,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.function.LongUnaryOperator; import static org.elasticsearch.compute.aggregation.AggregatorMode.FINAL; import static org.elasticsearch.compute.aggregation.AggregatorMode.INITIAL; @@ -231,13 +226,9 @@ public String toString() { }, new OrdinalsGroupingOperator( List.of( - new ValueSourceInfo( - CoreValuesSourceType.KEYWORD, - randomBoolean() ? getOrdinalsValuesSource(gField) : getBytesValuesSource(gField), - ElementType.BYTES_REF, - reader - ) + BlockReaderFactories.loaderToFactory(reader, new KeywordFieldMapper.KeywordFieldType("g").blockLoader(null)) ), + ElementType.BYTES_REF, 0, gField, List.of(CountAggregatorFunction.supplier(bigArrays, List.of(1)).groupingAggregatorFactory(INITIAL)), @@ -333,61 +324,6 @@ public ScoreMode scoreMode() { return docIds; } - static ValuesSource.Bytes.WithOrdinals getOrdinalsValuesSource(String field) { - return new ValuesSource.Bytes.WithOrdinals() { - - @Override - public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { - return getBytesValuesSource(field).bytesValues(context); - } - - @Override - public SortedSetDocValues ordinalsValues(LeafReaderContext context) throws IOException { - return context.reader().getSortedSetDocValues(field); - } - - @Override - public SortedSetDocValues globalOrdinalsValues(LeafReaderContext context) { - throw new UnsupportedOperationException(); - } - - @Override - public boolean supportsGlobalOrdinalsMapping() { - return false; - } - - @Override - public LongUnaryOperator globalOrdinalsMapping(LeafReaderContext context) { - throw new UnsupportedOperationException(); - } - }; - } - - static ValuesSource.Bytes getBytesValuesSource(String field) { - return new ValuesSource.Bytes() { - @Override - public SortedBinaryDocValues bytesValues(LeafReaderContext context) throws IOException { - final SortedSetDocValues dv = context.reader().getSortedSetDocValues(field); - return new SortedBinaryDocValues() { - @Override - public boolean advanceExact(int doc) throws IOException { - return dv.advanceExact(doc); - } - - @Override - public int docValueCount() { - return dv.docValueCount(); - } - - @Override - public BytesRef nextValue() throws IOException { - return dv.lookupOrd(dv.nextOrd()); - } - }; - } - }; - } - /** * Creates a {@link BigArrays} that tracks releases but doesn't throw circuit breaking exceptions. */ diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/SingletonOrdinalsBuilderTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/SingletonOrdinalsBuilderTests.java new file mode 100644 index 0000000000000..ff231a0cc20e0 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/SingletonOrdinalsBuilderTests.java @@ -0,0 +1,140 @@ +/* + * 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.apache.lucene.document.SortedDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedDocValues; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.indices.CrankyCircuitBreakerService; +import org.elasticsearch.test.ESTestCase; +import org.junit.After; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.test.ListMatcher.matchesList; +import static org.elasticsearch.test.MapMatcher.assertMap; +import static org.elasticsearch.test.MapMatcher.matchesMap; +import static org.hamcrest.Matchers.equalTo; + +public class SingletonOrdinalsBuilderTests extends ESTestCase { + public void testReader() throws IOException { + testRead(breakingDriverContext().blockFactory()); + } + + public void testReadWithCranky() throws IOException { + BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, new CrankyCircuitBreakerService()); + BlockFactory factory = new BlockFactory(bigArrays.breakerService().getBreaker(CircuitBreaker.REQUEST), bigArrays); + try { + testRead(factory); + // If we made it this far cranky didn't fail us! + } catch (CircuitBreakingException e) { + logger.info("cranky", e); + assertThat(e.getMessage(), equalTo(CrankyCircuitBreakerService.ERROR_MESSAGE)); + } + assertThat(factory.breaker().getUsed(), equalTo(0L)); + } + + private void testRead(BlockFactory factory) throws IOException { + int count = 1000; + try (Directory directory = newDirectory(); RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { + for (int i = 0; i < count; i++) { + for (BytesRef v : new BytesRef[] { new BytesRef("a"), new BytesRef("b"), new BytesRef("c"), new BytesRef("d") }) { + indexWriter.addDocument(List.of(new SortedDocValuesField("f", v))); + } + } + Map counts = new HashMap<>(); + try (IndexReader reader = indexWriter.getReader()) { + for (LeafReaderContext ctx : reader.leaves()) { + SortedDocValues docValues = ctx.reader().getSortedDocValues("f"); + try (SingletonOrdinalsBuilder builder = new SingletonOrdinalsBuilder(factory, docValues, ctx.reader().numDocs())) { + for (int i = 0; i < ctx.reader().maxDoc(); i++) { + if (ctx.reader().getLiveDocs() == null || ctx.reader().getLiveDocs().get(i)) { + assertThat(docValues.advanceExact(i), equalTo(true)); + builder.appendOrd(docValues.ordValue()); + } + } + try (BytesRefBlock build = builder.build()) { + for (int i = 0; i < build.getPositionCount(); i++) { + counts.merge(build.getBytesRef(i, new BytesRef()).utf8ToString(), 1, (lhs, rhs) -> lhs + rhs); + } + } + } + } + } + assertMap(counts, matchesMap().entry("a", count).entry("b", count).entry("c", count).entry("d", count)); + } + } + + public void testCompactWithNulls() { + assertCompactToUnique(new int[] { -1, -1, -1, -1, 0, 1, 2 }, List.of(0, 1, 2)); + } + + public void testCompactNoNulls() { + assertCompactToUnique(new int[] { 0, 1, 2 }, List.of(0, 1, 2)); + } + + public void testCompactDups() { + assertCompactToUnique(new int[] { 0, 0, 0, 1, 2 }, List.of(0, 1, 2)); + } + + public void testCompactSkips() { + assertCompactToUnique(new int[] { 2, 7, 1000 }, List.of(2, 7, 1000)); + } + + private void assertCompactToUnique(int[] sortedOrds, List expected) { + int uniqueLength = SingletonOrdinalsBuilder.compactToUnique(sortedOrds); + assertMap(Arrays.stream(sortedOrds).mapToObj(Integer::valueOf).limit(uniqueLength).toList(), matchesList(expected)); + } + + private final List breakers = new ArrayList<>(); + private final List blockFactories = new ArrayList<>(); + + /** + * A {@link DriverContext} with a breaking {@link BigArrays} and {@link BlockFactory}. + */ + protected DriverContext breakingDriverContext() { // TODO move this to driverContext once everyone supports breaking + BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, ByteSizeValue.ofGb(1)).withCircuitBreaking(); + CircuitBreaker breaker = bigArrays.breakerService().getBreaker(CircuitBreaker.REQUEST); + breakers.add(breaker); + BlockFactory factory = new MockBlockFactory(breaker, bigArrays); + blockFactories.add(factory); + return new DriverContext(bigArrays, factory); + } + + @After + public void allBreakersEmpty() throws Exception { + // first check that all big arrays are released, which can affect breakers + MockBigArrays.ensureAllArraysAreReleased(); + + for (CircuitBreaker breaker : breakers) { + for (var factory : blockFactories) { + if (factory instanceof MockBlockFactory mockBlockFactory) { + mockBlockFactory.ensureAllBlocksAreReleased(); + } + } + assertThat("Unexpected used in breaker: " + breaker, breaker.getUsed(), equalTo(0L)); + } + } + +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java index bbafc8ed753cc..131082859bf4c 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java @@ -17,7 +17,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.AnyOperatorTestCase; @@ -36,7 +35,6 @@ import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.query.support.NestedScope; -import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.search.internal.SearchContext; import org.junit.After; @@ -58,7 +56,7 @@ import static org.mockito.Mockito.when; public class LuceneSourceOperatorTests extends AnyOperatorTestCase { - private static final MappedFieldType S_FIELD = new NumberFieldMapper.NumberFieldType("s", NumberFieldMapper.NumberType.INTEGER); + private static final MappedFieldType S_FIELD = new NumberFieldMapper.NumberFieldType("s", NumberFieldMapper.NumberType.LONG); private Directory directory = newDirectory(); private IndexReader reader; @@ -145,12 +143,7 @@ public void testEmpty() { private void testSimple(int size, int limit) { DriverContext ctx = driverContext(); LuceneSourceOperator.Factory factory = simple(nonBreakingBigArrays(), DataPartitioning.SHARD, size, limit); - Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.LONG, - S_FIELD - ); + Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD); List results = new ArrayList<>(); OperatorTestCase.runDriver( diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java index 54853abd0cecb..f0eb49c233e7a 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java @@ -16,7 +16,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.AnyOperatorTestCase; @@ -34,7 +33,6 @@ import org.elasticsearch.index.mapper.NumberFieldMapper; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.query.support.NestedScope; -import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.SortBuilder; @@ -53,7 +51,7 @@ import static org.mockito.Mockito.when; public class LuceneTopNSourceOperatorTests extends AnyOperatorTestCase { - private static final MappedFieldType S_FIELD = new NumberFieldMapper.NumberFieldType("s", NumberFieldMapper.NumberType.INTEGER); + private static final MappedFieldType S_FIELD = new NumberFieldMapper.NumberFieldType("s", NumberFieldMapper.NumberType.LONG); private Directory directory = newDirectory(); private IndexReader reader; @@ -150,12 +148,7 @@ public void testEmpty() { private void testSimple(int size, int limit) { DriverContext ctx = driverContext(); LuceneTopNSourceOperator.Factory factory = simple(nonBreakingBigArrays(), DataPartitioning.SHARD, size, limit); - Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.LONG, - S_FIELD - ); + Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD); List results = new ArrayList<>(); OperatorTestCase.runDriver( diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java index ec1697e9aedd2..a6e5e3bf4744d 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java @@ -33,7 +33,6 @@ import org.elasticsearch.compute.data.BytesRefVector; import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.DoubleVector; -import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.LongBlock; @@ -47,18 +46,10 @@ import org.elasticsearch.compute.operator.PageConsumerOperator; import org.elasticsearch.compute.operator.SourceOperator; import org.elasticsearch.core.IOUtils; -import org.elasticsearch.index.fielddata.FieldDataContext; -import org.elasticsearch.index.fielddata.IndexFieldData; -import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.mapper.BooleanFieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.NumberFieldMapper; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; -import org.elasticsearch.search.aggregations.support.FieldContext; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceType; import org.junit.After; import java.io.IOException; @@ -95,21 +86,12 @@ public void closeIndex() throws IOException { @Override protected Operator.OperatorFactory simple(BigArrays bigArrays) { - return factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.LONG, - new NumberFieldMapper.NumberFieldType("long", NumberFieldMapper.NumberType.LONG) - ); + return factory(reader, new NumberFieldMapper.NumberFieldType("long", NumberFieldMapper.NumberType.LONG)); } - static Operator.OperatorFactory factory(IndexReader reader, ValuesSourceType vsType, ElementType elementType, MappedFieldType ft) { - IndexFieldData fd = ft.fielddataBuilder(FieldDataContext.noRuntimeFields("test")) - .build(new IndexFieldDataCache.None(), new NoneCircuitBreakerService()); - FieldContext fc = new FieldContext(ft.name(), fd, ft); - ValuesSource vs = vsType.getField(fc, null); + static Operator.OperatorFactory factory(IndexReader reader, MappedFieldType ft) { return new ValuesSourceReaderOperator.ValuesSourceReaderOperatorFactory( - () -> List.of(new ValueSourceInfo(vsType, vs, elementType, reader)), + List.of(BlockReaderFactories.loaderToFactory(reader, ft.blockLoader(null))), 0, ft.name() ); @@ -243,54 +225,16 @@ public void testLoadAllInOnePageShuffled() { private void loadSimpleAndAssert(DriverContext driverContext, List input) { List results = new ArrayList<>(); List operators = List.of( - factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.INT, - new NumberFieldMapper.NumberFieldType("key", NumberFieldMapper.NumberType.INTEGER) - ).get(driverContext), - factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.LONG, - new NumberFieldMapper.NumberFieldType("long", NumberFieldMapper.NumberType.LONG) - ).get(driverContext), - factory(reader, CoreValuesSourceType.KEYWORD, ElementType.BYTES_REF, new KeywordFieldMapper.KeywordFieldType("kwd")).get( - driverContext - ), - factory(reader, CoreValuesSourceType.KEYWORD, ElementType.BYTES_REF, new KeywordFieldMapper.KeywordFieldType("mv_kwd")).get( - driverContext - ), - factory(reader, CoreValuesSourceType.BOOLEAN, ElementType.BOOLEAN, new BooleanFieldMapper.BooleanFieldType("bool")).get( - driverContext - ), - factory(reader, CoreValuesSourceType.BOOLEAN, ElementType.BOOLEAN, new BooleanFieldMapper.BooleanFieldType("mv_bool")).get( - driverContext - ), - factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.INT, - new NumberFieldMapper.NumberFieldType("mv_key", NumberFieldMapper.NumberType.INTEGER) - ).get(driverContext), - factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.LONG, - new NumberFieldMapper.NumberFieldType("mv_long", NumberFieldMapper.NumberType.LONG) - ).get(driverContext), - factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.DOUBLE, - new NumberFieldMapper.NumberFieldType("double", NumberFieldMapper.NumberType.DOUBLE) - ).get(driverContext), - factory( - reader, - CoreValuesSourceType.NUMERIC, - ElementType.DOUBLE, - new NumberFieldMapper.NumberFieldType("mv_double", NumberFieldMapper.NumberType.DOUBLE) - ).get(driverContext) + factory(reader, new NumberFieldMapper.NumberFieldType("key", NumberFieldMapper.NumberType.INTEGER)).get(driverContext), + factory(reader, new NumberFieldMapper.NumberFieldType("long", NumberFieldMapper.NumberType.LONG)).get(driverContext), + factory(reader, new KeywordFieldMapper.KeywordFieldType("kwd")).get(driverContext), + factory(reader, new KeywordFieldMapper.KeywordFieldType("mv_kwd")).get(driverContext), + factory(reader, new BooleanFieldMapper.BooleanFieldType("bool")).get(driverContext), + factory(reader, new BooleanFieldMapper.BooleanFieldType("mv_bool")).get(driverContext), + factory(reader, new NumberFieldMapper.NumberFieldType("mv_key", NumberFieldMapper.NumberType.INTEGER)).get(driverContext), + factory(reader, new NumberFieldMapper.NumberFieldType("mv_long", NumberFieldMapper.NumberType.LONG)).get(driverContext), + factory(reader, new NumberFieldMapper.NumberFieldType("double", NumberFieldMapper.NumberType.DOUBLE)).get(driverContext), + factory(reader, new NumberFieldMapper.NumberFieldType("mv_double", NumberFieldMapper.NumberType.DOUBLE)).get(driverContext) ); try ( Driver d = new Driver( @@ -418,10 +362,10 @@ public void testValuesSourceReaderOperatorWithNulls() throws IOException { driverContext, luceneFactory.get(driverContext), List.of( - factory(reader, CoreValuesSourceType.NUMERIC, ElementType.INT, intFt).get(driverContext), - factory(reader, CoreValuesSourceType.NUMERIC, ElementType.LONG, longFt).get(driverContext), - factory(reader, CoreValuesSourceType.NUMERIC, ElementType.DOUBLE, doubleFt).get(driverContext), - factory(reader, CoreValuesSourceType.KEYWORD, ElementType.BYTES_REF, kwFt).get(driverContext) + factory(reader, intFt).get(driverContext), + factory(reader, longFt).get(driverContext), + factory(reader, doubleFt).get(driverContext), + factory(reader, kwFt).get(driverContext) ), new PageConsumerOperator(page -> { logger.debug("New page: {}", page); diff --git a/x-pack/plugin/esql/qa/server/single-node/build.gradle b/x-pack/plugin/esql/qa/server/single-node/build.gradle index 792059639aaca..3131b4176ee25 100644 --- a/x-pack/plugin/esql/qa/server/single-node/build.gradle +++ b/x-pack/plugin/esql/qa/server/single-node/build.gradle @@ -7,7 +7,7 @@ dependencies { restResources { restApi { - include '_common', 'bulk', 'indices', 'esql', 'xpack', 'enrich', 'cluster' + include '_common', 'bulk', 'get', 'indices', 'esql', 'xpack', 'enrich', 'cluster' } } diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/30_types.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/30_types.yml index 886bb6dc60aca..7ec0f671253b9 100644 --- a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/30_types.yml +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/30_types.yml @@ -112,7 +112,7 @@ keyword no doc_values: - match: {columns.0.name: card} - match: {columns.0.type: keyword} - length: {values: 1} - - match: {values.0.0: [diamonds, jack, of]} + - match: {values.0.0: [jack, of, diamonds]} --- wildcard: diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_tsdb.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_tsdb.yml index 33697a789cc26..6a90fc5a7b8f8 100644 --- a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_tsdb.yml +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/40_tsdb.yml @@ -111,7 +111,7 @@ load everything: - "No limit defined, adding default limit of \\[.*\\]" esql.query: body: - query: 'from test' + query: 'from test [metadata _id]' - match: {columns.0.name: "@timestamp"} - match: {columns.0.type: "date"} diff --git a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/90_non_indexed.yml b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/90_non_indexed.yml index a673fb7a5b88d..9138a9454c571 100644 --- a/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/90_non_indexed.yml +++ b/x-pack/plugin/esql/qa/server/single-node/src/yamlRestTest/resources/rest-api-spec/test/90_non_indexed.yml @@ -1,6 +1,6 @@ setup: - skip: - features: allowed_warnings_regex + features: allowed_warnings - do: indices.create: index: test @@ -85,11 +85,11 @@ setup: } --- -unsupported: +fetch: - do: - allowed_warnings_regex: - - "Field \\[.*\\] cannot be retrieved, it is unsupported or not indexed; returning null" - - "No limit defined, adding default limit of \\[.*\\]" + allowed_warnings: + - "Field [ip_noidx] cannot be retrieved, it is unsupported or not indexed; returning null" + - "No limit defined, adding default limit of [500]" esql.query: body: query: 'from test' @@ -130,18 +130,18 @@ unsupported: - length: { values: 1 } - match: { values.0.0: true } - - match: { values.0.1: null } + - match: { values.0.1: true } - match: { values.0.2: "2021-04-28T18:50:04.467Z" } - - match: { values.0.3: null } + - match: { values.0.3: "2021-04-28T18:50:04.467Z" } - match: { values.0.4: 40 } - - match: { values.0.5: null } + - match: { values.0.5: 40 } - match: { values.0.6: 30 } - - match: { values.0.7: null } + - match: { values.0.7: 30 } - match: { values.0.8: 10 } - - match: { values.0.9: null } + - match: { values.0.9: 10 } - match: { values.0.10: "192.168.0.1" } - match: { values.0.11: null } - match: { values.0.12: "foo" } - - match: { values.0.13: "foo" } # this is a special case, ESQL can retrieve keywords from source + - match: { values.0.13: "foo" } - match: { values.0.14: 20 } - - match: { values.0.15: null } + - match: { values.0.15: 20 } diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java index 6ebd1b24c13bd..1729c9fc363e4 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.ListMatcher; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.json.JsonXContent; import org.elasticsearch.xpack.esql.analysis.VerificationException; @@ -50,6 +51,8 @@ import static java.util.Comparator.comparing; import static java.util.Comparator.naturalOrder; import static java.util.Comparator.reverseOrder; +import static org.elasticsearch.test.ListMatcher.matchesList; +import static org.elasticsearch.test.MapMatcher.assertMap; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.xpack.esql.EsqlTestUtils.getValuesList; import static org.hamcrest.Matchers.allOf; @@ -81,7 +84,6 @@ public void testProjectConstant() { assertThat(getValuesList(results).size(), equalTo(40)); assertThat(getValuesList(results).get(0).get(0), equalTo(1)); } - } public void testStatsOverConstant() { @@ -1180,6 +1182,17 @@ public void testGroupingMultiValueByOrdinals() { } } + public void testLoadId() { + try (EsqlQueryResponse results = run("from test [metadata _id] | keep _id | sort _id ")) { + assertThat(results.columns(), equalTo(List.of(new ColumnInfo("_id", "keyword")))); + ListMatcher values = matchesList(); + for (int i = 10; i < 50; i++) { + values = values.item(List.of(Integer.toString(i))); + } + assertMap(getValuesList(results), values); + } + } + public void testUnsupportedTypesOrdinalGrouping() { assertAcked( client().admin().indices().prepareCreate("index-1").setMapping("f1", "type=keyword", "f2", "type=keyword", "v", "type=long") diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionRuntimeFieldIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionRuntimeFieldIT.java index 41450be131e2a..be661b51d41d5 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionRuntimeFieldIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionRuntimeFieldIT.java @@ -38,8 +38,7 @@ import static org.hamcrest.Matchers.equalTo; /** - * Makes sure that the circuit breaker is "plugged in" to ESQL by configuring an - * unreasonably small breaker and tripping it. + * Tests runtime fields against ESQL. */ @ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) // @TestLogging(value = "org.elasticsearch.xpack.esql:TRACE", reason = "debug") diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java index b1fab0ab94af9..edaf9d91e9771 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java @@ -177,7 +177,7 @@ public void testTaskContents() throws Exception { } if (o.operator().equals("ValuesSourceReaderOperator[field = pause_me]")) { ValuesSourceReaderOperator.Status oStatus = (ValuesSourceReaderOperator.Status) o.status(); - assertMap(oStatus.readersBuilt(), matchesMap().entry("LongValuesReader", greaterThanOrEqualTo(1))); + assertMap(oStatus.readersBuilt(), matchesMap().entry("ScriptLongs", greaterThanOrEqualTo(1))); assertThat(oStatus.pagesProcessed(), greaterThanOrEqualTo(1)); valuesSourceReaders++; continue; diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/SyntheticSourceIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/SyntheticSourceIT.java index f0365ce78f44a..2585b5325df18 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/SyntheticSourceIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/SyntheticSourceIT.java @@ -9,11 +9,13 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.index.mapper.extras.MapperExtrasPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.json.JsonXContent; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; @@ -30,40 +32,64 @@ protected Collection> nodePlugins() { } public void testMatchOnlyText() throws Exception { - XContentBuilder mapping = JsonXContent.contentBuilder(); - mapping.startObject(); - if (true || randomBoolean()) { - mapping.startObject("_source"); - mapping.field("mode", "synthetic"); - mapping.endObject(); + createIndex(b -> b.field("type", "match_only_text")); + + int numDocs = between(10, 1000); + for (int i = 0; i < numDocs; i++) { + IndexRequestBuilder indexRequest = client().prepareIndex("test").setSource("id", "i" + i, "field", "n" + i); + if (randomInt(100) < 5) { + indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + } + indexRequest.get(); } - { - mapping.startObject("properties"); - mapping.startObject("uid"); - mapping.field("type", "keyword"); - mapping.endObject(); - mapping.startObject("name"); - mapping.field("type", "match_only_text"); - mapping.endObject(); - mapping.endObject(); + client().admin().indices().prepareRefresh("test").get(); + + try (EsqlQueryResponse resp = run("from test | sort id asc | limit 1")) { + Iterator row = resp.values().next(); + assertThat(row.next(), equalTo("n0")); + assertThat(row.next(), equalTo("i0")); + assertFalse(row.hasNext()); } - mapping.endObject(); + } - assertAcked(client().admin().indices().prepareCreate("test").setMapping(mapping)); + public void testText() throws Exception { + createIndex(b -> b.field("type", "text").field("store", true)); int numDocs = between(10, 1000); for (int i = 0; i < numDocs; i++) { - IndexRequestBuilder indexRequest = client().prepareIndex("test").setSource("uid", "u" + i); + IndexRequestBuilder indexRequest = client().prepareIndex("test").setSource("id", "i" + i, "field", "n" + i); if (randomInt(100) < 5) { indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); } indexRequest.get(); } client().admin().indices().prepareRefresh("test").get(); - try (EsqlQueryResponse resp = run("from test | keep uid, name | sort uid asc | limit 1")) { + try (EsqlQueryResponse resp = run("from test | keep field, id | sort id asc | limit 1")) { Iterator row = resp.values().next(); - assertThat(row.next(), equalTo("u0")); - assertNull(row.next()); + assertThat(row.next(), equalTo("n0")); + assertThat(row.next(), equalTo("i0")); + assertFalse(row.hasNext()); + } + } + + private void createIndex(CheckedFunction fieldMapping) throws IOException { + XContentBuilder mapping = JsonXContent.contentBuilder(); + mapping.startObject(); + { + mapping.startObject("_source"); + mapping.field("mode", "synthetic"); + mapping.endObject(); + } + { + mapping.startObject("properties"); + mapping.startObject("id").field("type", "keyword").endObject(); + mapping.startObject("field"); + fieldMapping.apply(mapping); + mapping.endObject(); + mapping.endObject(); } + mapping.endObject(); + + assertAcked(client().admin().indices().prepareCreate("test").setMapping(mapping)); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java index af8732ad9c969..bf246b5ac02d4 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/EnrichLookupService.java @@ -30,7 +30,7 @@ import org.elasticsearch.compute.data.BlockStreamInput; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.Page; -import org.elasticsearch.compute.lucene.ValueSources; +import org.elasticsearch.compute.lucene.BlockReaderFactories; import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; import org.elasticsearch.compute.operator.Driver; import org.elasticsearch.compute.operator.DriverContext; @@ -250,11 +250,10 @@ private void doLookup( NamedExpression extractField = extractFields.get(i); final ElementType elementType = LocalExecutionPlanner.toElementType(extractField.dataType()); mergingTypes[i] = elementType; - var sources = ValueSources.sources( + var sources = BlockReaderFactories.factories( List.of(searchContext), extractField instanceof Alias a ? ((NamedExpression) a.child()).name() : extractField.name(), - EsqlDataTypes.isUnsupported(extractField.dataType()), - elementType + EsqlDataTypes.isUnsupported(extractField.dataType()) ); intermediateOperators.add(new ValuesSourceReaderOperator(sources, 0, extractField.name())); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java index 3131b8c8c1e20..2cb739d6f068e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java @@ -12,21 +12,19 @@ import org.apache.lucene.search.Query; import org.elasticsearch.compute.aggregation.GroupingAggregator; import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.lucene.BlockReaderFactories; import org.elasticsearch.compute.lucene.LuceneOperator; import org.elasticsearch.compute.lucene.LuceneSourceOperator; import org.elasticsearch.compute.lucene.LuceneTopNSourceOperator; -import org.elasticsearch.compute.lucene.ValueSourceInfo; -import org.elasticsearch.compute.lucene.ValueSources; import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; import org.elasticsearch.compute.operator.Operator; import org.elasticsearch.compute.operator.OrdinalsGroupingOperator; +import org.elasticsearch.index.mapper.BlockDocValuesReader; import org.elasticsearch.index.mapper.NestedLookup; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.search.NestedHelper; -import org.elasticsearch.logging.LogManager; -import org.elasticsearch.logging.Logger; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.sort.SortBuilder; @@ -45,13 +43,11 @@ import java.util.ArrayList; import java.util.List; import java.util.function.Function; -import java.util.function.Supplier; import static org.elasticsearch.common.lucene.search.Queries.newNonNestedFilter; import static org.elasticsearch.compute.lucene.LuceneSourceOperator.NO_LIMIT; public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProviders { - private static final Logger logger = LogManager.getLogger(EsPhysicalOperationProviders.class); private final List searchContexts; @@ -79,16 +75,18 @@ public final PhysicalOperation fieldExtractPhysicalOperation(FieldExtractExec fi DataType dataType = attr.dataType(); String fieldName = attr.name(); - Supplier> sources = () -> ValueSources.sources( + List factories = BlockReaderFactories.factories( searchContexts, fieldName, - EsqlDataTypes.isUnsupported(dataType), - LocalExecutionPlanner.toElementType(dataType) + EsqlDataTypes.isUnsupported(dataType) ); int docChannel = previousLayout.get(sourceAttr.id()).channel(); - op = op.with(new ValuesSourceReaderOperator.ValuesSourceReaderOperatorFactory(sources, docChannel, fieldName), layout.build()); + op = op.with( + new ValuesSourceReaderOperator.ValuesSourceReaderOperatorFactory(factories, docChannel, fieldName), + layout.build() + ); } return op; } @@ -175,12 +173,8 @@ public final Operator.OperatorFactory ordinalGroupingOperatorFactory( // The grouping-by values are ready, let's group on them directly. // Costin: why are they ready and not already exposed in the layout? return new OrdinalsGroupingOperator.OrdinalsGroupingOperatorFactory( - () -> ValueSources.sources( - searchContexts, - attrSource.name(), - EsqlDataTypes.isUnsupported(attrSource.dataType()), - LocalExecutionPlanner.toElementType(attrSource.dataType()) - ), + BlockReaderFactories.factories(searchContexts, attrSource.name(), EsqlDataTypes.isUnsupported(attrSource.dataType())), + groupElementType, docChannel, attrSource.name(), aggregatorFactories, diff --git a/x-pack/plugin/mapper-constant-keyword/src/main/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapper.java b/x-pack/plugin/mapper-constant-keyword/src/main/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapper.java index 9723276c827fc..75202749d8dca 100644 --- a/x-pack/plugin/mapper-constant-keyword/src/main/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapper.java +++ b/x-pack/plugin/mapper-constant-keyword/src/main/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapper.java @@ -30,6 +30,8 @@ import org.elasticsearch.index.fielddata.FieldDataContext; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.plain.ConstantIndexFieldData; +import org.elasticsearch.index.mapper.BlockDocValuesReader; +import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.ConstantFieldType; import org.elasticsearch.index.mapper.DocumentParserContext; import org.elasticsearch.index.mapper.FieldMapper; @@ -133,6 +135,45 @@ public String familyTypeName() { return KeywordFieldMapper.CONTENT_TYPE; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + // TODO build a constant block directly + BytesRef bytes = new BytesRef(value); + return context -> new BlockDocValuesReader() { + private int docId; + + @Override + public int docID() { + return docId; + } + + @Override + public BlockLoader.BytesRefBuilder builder(BlockLoader.BuilderFactory factory, int expectedCount) { + return factory.bytesRefs(expectedCount); + } + + @Override + public BlockLoader.Builder readValues(BlockLoader.BuilderFactory factory, BlockLoader.Docs docs) { + BlockLoader.BytesRefBuilder builder = builder(factory, docs.count()); + for (int i = 0; i < docs.count(); i++) { + builder.appendBytesRef(bytes); + } + return builder; + } + + @Override + public void readValuesFromSingleDoc(int docId, BlockLoader.Builder builder) { + this.docId = docId; + ((BlockLoader.BytesRefBuilder) builder).appendBytesRef(bytes); + } + + @Override + public String toString() { + return "ConstantKeyword"; + } + }; + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { return new ConstantIndexFieldData.Builder( diff --git a/x-pack/plugin/mapper-constant-keyword/src/test/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapperTests.java b/x-pack/plugin/mapper-constant-keyword/src/test/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapperTests.java index 859168e154ff8..5fb8f3f7dc345 100644 --- a/x-pack/plugin/mapper-constant-keyword/src/test/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapperTests.java +++ b/x-pack/plugin/mapper-constant-keyword/src/test/java/org/elasticsearch/xpack/constantkeyword/mapper/ConstantKeywordFieldMapperTests.java @@ -9,6 +9,7 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.Settings; @@ -30,6 +31,7 @@ import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.function.Function; import static org.elasticsearch.index.mapper.MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING; import static org.hamcrest.Matchers.equalTo; @@ -237,6 +239,11 @@ protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); } + @Override + protected Function loadBlockExpected() { + return v -> ((BytesRef) v).utf8ToString(); + } + public void testNullValueSyntheticSource() throws IOException { DocumentMapper mapper = createDocumentMapper(syntheticSourceMapping(b -> { b.startObject("field"); diff --git a/x-pack/plugin/mapper-unsigned-long/src/main/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapper.java b/x-pack/plugin/mapper-unsigned-long/src/main/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapper.java index ca23799300f24..62b02f5a3d850 100644 --- a/x-pack/plugin/mapper-unsigned-long/src/main/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapper.java +++ b/x-pack/plugin/mapper-unsigned-long/src/main/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapper.java @@ -24,6 +24,9 @@ import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexNumericFieldData; import org.elasticsearch.index.fielddata.plain.SortedNumericIndexFieldData; +import org.elasticsearch.index.mapper.BlockDocValuesReader; +import org.elasticsearch.index.mapper.BlockLoader; +import org.elasticsearch.index.mapper.BlockSourceReader; import org.elasticsearch.index.mapper.DocumentParserContext; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; @@ -313,6 +316,26 @@ public Query rangeQuery( return query; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (indexMode == IndexMode.TIME_SERIES && metricType == TimeSeriesParams.MetricType.COUNTER) { + // Counters are not supported by ESQL so we load them in null + return BlockDocValuesReader.nulls(); + } + if (hasDocValues()) { + return BlockDocValuesReader.longs(name()); + } + return BlockSourceReader.longs(new SourceValueFetcher(blContext.sourcePaths(name()), nullValueFormatted) { + @Override + protected Object parseSourceValue(Object value) { + if (value.equals("")) { + return nullValueFormatted; + } + return parseUnsignedLong(value); + } + }); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { FielddataOperation operation = fieldDataContext.fielddataOperation(); diff --git a/x-pack/plugin/mapper-unsigned-long/src/test/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapperTests.java b/x-pack/plugin/mapper-unsigned-long/src/test/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapperTests.java index 54a1edd88eb6b..95fe8f0a530ba 100644 --- a/x-pack/plugin/mapper-unsigned-long/src/test/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapperTests.java +++ b/x-pack/plugin/mapper-unsigned-long/src/test/java/org/elasticsearch/xpack/unsignedlong/UnsignedLongFieldMapperTests.java @@ -30,6 +30,7 @@ import java.math.BigInteger; import java.util.Collection; import java.util.List; +import java.util.function.Function; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; @@ -365,6 +366,20 @@ protected IngestScriptSupport ingestScriptSupport() { throw new AssumptionViolatedException("not supported"); } + @Override + protected Function loadBlockExpected() { + return v -> { + // Numbers are in the block as a long but the test needs to compare them to their BigInteger value parsed from xcontent. + if (v instanceof BigInteger ul) { + if (ul.bitLength() < Long.SIZE) { + return ul.longValue() ^ Long.MIN_VALUE; + } + return ul.subtract(BigInteger.ONE.shiftLeft(Long.SIZE - 1)).longValue(); + } + return ((Long) v).longValue() ^ Long.MIN_VALUE; + }; + } + final class NumberSyntheticSourceSupport implements SyntheticSourceSupport { private final BigInteger nullValue = usually() ? null : BigInteger.valueOf(randomNonNegativeLong()); diff --git a/x-pack/plugin/mapper-version/src/main/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapper.java b/x-pack/plugin/mapper-version/src/main/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapper.java index 9e88c516576c2..f4fb83fd9a91c 100644 --- a/x-pack/plugin/mapper-version/src/main/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapper.java +++ b/x-pack/plugin/mapper-version/src/main/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapper.java @@ -39,6 +39,8 @@ import org.elasticsearch.index.fielddata.FieldDataContext; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.plain.SortedSetOrdinalsIndexFieldData; +import org.elasticsearch.index.mapper.BlockDocValuesReader; +import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.DocumentParserContext; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; @@ -290,6 +292,12 @@ protected BytesRef indexedValueForSearch(Object value) { return encodeVersion(valueAsString).bytesRef; } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + failIfNoDocValues(); + return BlockDocValuesReader.bytesRefsFromOrds(name()); + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { return new SortedSetOrdinalsIndexFieldData.Builder(name(), CoreValuesSourceType.KEYWORD, VersionStringDocValuesField::new); diff --git a/x-pack/plugin/mapper-version/src/test/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapperTests.java b/x-pack/plugin/mapper-version/src/test/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapperTests.java index d94a632ff9787..5653ed7f4302f 100644 --- a/x-pack/plugin/mapper-version/src/test/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapperTests.java +++ b/x-pack/plugin/mapper-version/src/test/java/org/elasticsearch/xpack/versionfield/VersionStringFieldMapperTests.java @@ -11,6 +11,7 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.IndexableFieldType; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.Tuple; @@ -31,6 +32,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.function.Function; import java.util.stream.Collectors; import static org.hamcrest.Matchers.equalTo; @@ -185,6 +187,11 @@ protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) return new VersionStringSyntheticSourceSupport(); } + @Override + protected Function loadBlockExpected() { + return v -> new Version((BytesRef) v).toString(); + } + static class VersionStringSyntheticSourceSupport implements SyntheticSourceSupport { @Override public SyntheticSourceExample example(int maxValues) { diff --git a/x-pack/plugin/wildcard/src/main/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapper.java b/x-pack/plugin/wildcard/src/main/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapper.java index 4b4dea8829791..ec206c64a2371 100644 --- a/x-pack/plugin/wildcard/src/main/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapper.java +++ b/x-pack/plugin/wildcard/src/main/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapper.java @@ -60,6 +60,8 @@ import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.plain.StringBinaryIndexFieldData; import org.elasticsearch.index.mapper.BinaryFieldMapper.CustomBinaryDocValuesField; +import org.elasticsearch.index.mapper.BlockDocValuesReader; +import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.DocumentParserContext; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper; @@ -851,6 +853,16 @@ public Query termsQuery(Collection values, SearchExecutionContext context) { return new ConstantScoreQuery(bq.build()); } + @Override + public BlockLoader blockLoader(BlockLoaderContext blContext) { + if (hasDocValues()) { + // TODO it'd almost certainly be faster to drop directly to doc values like we do with keyword but this'll do for now + IndexFieldData fd = new StringBinaryIndexFieldData(name(), CoreValuesSourceType.KEYWORD, null); + return BlockDocValuesReader.bytesRefsFromDocValues(context -> fd.load(context).getBytesValues()); + } + return null; + } + @Override public IndexFieldData.Builder fielddataBuilder(FieldDataContext fieldDataContext) { failIfNoDocValues(); diff --git a/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java index 1fade663cbe2d..a17cb7474a681 100644 --- a/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java +++ b/x-pack/plugin/wildcard/src/test/java/org/elasticsearch/xpack/wildcard/mapper/WildcardFieldMapperTests.java @@ -83,6 +83,7 @@ import java.util.HashSet; import java.util.List; import java.util.function.BiFunction; +import java.util.function.Function; import static java.util.Collections.emptyMap; import static org.hamcrest.Matchers.equalTo; @@ -1215,6 +1216,11 @@ protected SyntheticSourceSupport syntheticSourceSupport(boolean ignoreMalformed) return new WildcardSyntheticSourceSupport(); } + @Override + protected Function loadBlockExpected() { + return v -> ((BytesRef) v).utf8ToString(); + } + static class WildcardSyntheticSourceSupport implements SyntheticSourceSupport { private final Integer ignoreAbove = randomBoolean() ? null : between(10, 100); private final boolean allIgnored = ignoreAbove != null && rarely(); @@ -1224,7 +1230,11 @@ static class WildcardSyntheticSourceSupport implements SyntheticSourceSupport { public SyntheticSourceExample example(int maxValues) { if (randomBoolean()) { Tuple v = generateValue(); - return new SyntheticSourceExample(v.v1(), v.v2(), this::mapping); + Object loadBlock = v.v2(); + if (ignoreAbove != null && v.v2().length() > ignoreAbove) { + loadBlock = null; + } + return new SyntheticSourceExample(v.v1(), v.v2(), loadBlock, this::mapping); } List> values = randomList(1, maxValues, this::generateValue); List in = values.stream().map(Tuple::v1).toList(); @@ -1239,9 +1249,11 @@ public SyntheticSourceExample example(int maxValues) { }); List outList = new ArrayList<>(new HashSet<>(docValuesValues)); Collections.sort(outList); + List outBlockList = List.copyOf(outList); + Object outBlockResult = outBlockList.size() == 1 ? outBlockList.get(0) : outBlockList; outList.addAll(outExtraValues); Object out = outList.size() == 1 ? outList.get(0) : outList; - return new SyntheticSourceExample(in, out, this::mapping); + return new SyntheticSourceExample(in, out, outBlockResult, this::mapping); } private Tuple generateValue() {