diff --git a/x-pack/plugin/mapper-exponential-histogram/build.gradle b/x-pack/plugin/mapper-exponential-histogram/build.gradle index fc262d693e896..e7e89b54583b9 100644 --- a/x-pack/plugin/mapper-exponential-histogram/build.gradle +++ b/x-pack/plugin/mapper-exponential-histogram/build.gradle @@ -18,6 +18,11 @@ base { archivesName = 'x-pack-exponential-histogram' } +restResources { + restApi { + include 'bulk', 'search' + } +} dependencies { api project(":libs:exponential-histogram") compileOnly project(path: xpackModule('core')) diff --git a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramFieldMapper.java b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramFieldMapper.java index 1f9078f979e72..b5f68ba1d875d 100644 --- a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramFieldMapper.java +++ b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramFieldMapper.java @@ -8,9 +8,9 @@ package org.elasticsearch.xpack.exponentialhistogram; import org.apache.lucene.document.BinaryDocValuesField; -import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; @@ -36,6 +36,7 @@ import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.IgnoreMalformedStoredValues; import org.elasticsearch.index.mapper.IndexType; +import org.elasticsearch.index.mapper.LuceneDocument; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MapperBuilderContext; import org.elasticsearch.index.mapper.SourceLoader; @@ -238,9 +239,10 @@ protected void parseCreateField(DocumentParserContext context) { throw new UnsupportedOperationException("Parsing is implemented in parse(), this method should NEVER be called"); } - static class ExponentialHistogramFieldType extends MappedFieldType { + public static final class ExponentialHistogramFieldType extends MappedFieldType { - ExponentialHistogramFieldType(String name, Map meta) { + // Visible for testing + public ExponentialHistogramFieldType(String name, Map meta) { super(name, IndexType.docValuesOnly(), false, meta); } @@ -445,37 +447,18 @@ public void parse(DocumentParserContext context) throws IOException { min = validateOrEstimateMin(min, zeroBucket, scale, negativeBuckets, positiveBuckets, totalValueCount, subParser); max = validateOrEstimateMax(max, zeroBucket, scale, negativeBuckets, positiveBuckets, totalValueCount, subParser); - BytesStreamOutput histogramBytesOutput = new BytesStreamOutput(); - CompressedExponentialHistogram.writeHistogramBytes(histogramBytesOutput, scale, negativeBuckets, positiveBuckets); - BytesRef histoBytes = histogramBytesOutput.bytes().toBytesRef(); - - Field histoField = new BinaryDocValuesField(fullPath(), histoBytes); - long thresholdAsLong = NumericUtils.doubleToSortableLong(zeroBucket.threshold()); - NumericDocValuesField zeroThresholdField = new NumericDocValuesField(zeroThresholdSubFieldName(fullPath()), thresholdAsLong); - NumericDocValuesField valuesCountField = new NumericDocValuesField(valuesCountSubFieldName(fullPath()), totalValueCount); - NumericDocValuesField sumField = new NumericDocValuesField( - valuesSumSubFieldName(fullPath()), - NumericUtils.doubleToSortableLong(sum) + HistogramDocValueFields docValues = buildDocValueFields( + fullPath(), + scale, + negativeBuckets, + positiveBuckets, + zeroBucket.threshold(), + totalValueCount, + sum, + min, + max ); - - context.doc().addWithKey(fieldType().name(), histoField); - context.doc().add(zeroThresholdField); - context.doc().add(valuesCountField); - context.doc().add(sumField); - if (min != null) { - NumericDocValuesField minField = new NumericDocValuesField( - valuesMinSubFieldName(fullPath()), - NumericUtils.doubleToSortableLong(min) - ); - context.doc().add(minField); - } - if (max != null) { - NumericDocValuesField maxField = new NumericDocValuesField( - valuesMaxSubFieldName(fullPath()), - NumericUtils.doubleToSortableLong(max) - ); - context.doc().add(maxField); - } + docValues.addToDoc(context.doc()); } catch (Exception ex) { if (ignoreMalformed.value() == false) { @@ -505,6 +488,88 @@ public void parse(DocumentParserContext context) throws IOException { context.path().remove(); } + // Visible for testing, to construct realistic doc values in tests + public static HistogramDocValueFields buildDocValueFields( + String fieldName, + int scale, + List negativeBuckets, + List positiveBuckets, + double zeroThreshold, + long totalValueCount, + double sum, + @Nullable Double min, + @Nullable Double max + ) throws IOException { + BytesStreamOutput histogramBytesOutput = new BytesStreamOutput(); + CompressedExponentialHistogram.writeHistogramBytes(histogramBytesOutput, scale, negativeBuckets, positiveBuckets); + BytesRef histoBytes = histogramBytesOutput.bytes().toBytesRef(); + + BinaryDocValuesField histoField = new BinaryDocValuesField(fieldName, histoBytes); + long thresholdAsLong = NumericUtils.doubleToSortableLong(zeroThreshold); + NumericDocValuesField zeroThresholdField = new NumericDocValuesField(zeroThresholdSubFieldName(fieldName), thresholdAsLong); + NumericDocValuesField valuesCountField = new NumericDocValuesField(valuesCountSubFieldName(fieldName), totalValueCount); + NumericDocValuesField sumField = new NumericDocValuesField( + valuesSumSubFieldName(fieldName), + NumericUtils.doubleToSortableLong(sum) + ); + NumericDocValuesField minField = null; + if (min != null) { + minField = new NumericDocValuesField(valuesMinSubFieldName(fieldName), NumericUtils.doubleToSortableLong(min)); + } + NumericDocValuesField maxField = null; + if (max != null) { + maxField = new NumericDocValuesField(valuesMaxSubFieldName(fieldName), NumericUtils.doubleToSortableLong(max)); + } + HistogramDocValueFields docValues = new HistogramDocValueFields( + histoField, + zeroThresholdField, + valuesCountField, + sumField, + minField, + maxField + ); + return docValues; + } + + // Visible for testing + public record HistogramDocValueFields( + BinaryDocValuesField histo, + NumericDocValuesField zeroThreshold, + NumericDocValuesField valuesCount, + NumericDocValuesField sumField, + @Nullable NumericDocValuesField minField, + @Nullable NumericDocValuesField maxField + ) { + + public void addToDoc(LuceneDocument doc) { + doc.addWithKey(histo.name(), histo); + doc.add(zeroThreshold); + doc.add(valuesCount); + doc.add(sumField); + if (minField != null) { + doc.add(minField); + } + if (maxField != null) { + doc.add(maxField); + } + } + + public List fieldsAsList() { + List fields = new ArrayList<>(); + fields.add(histo); + fields.add(zeroThreshold); + fields.add(valuesCount); + fields.add(sumField); + if (minField != null) { + fields.add(minField); + } + if (maxField != null) { + fields.add(maxField); + } + return fields; + } + } + private Double validateOrEstimateSum( Double sum, Integer scale, @@ -819,12 +884,12 @@ private static class DocValuesReader implements ExponentialHistogramValuesReader } boolean hasAnyValues() { - return histoDocValues != null; + return valueCounts != null; } @Override public boolean advanceExact(int docId) throws IOException { - boolean isPresent = histoDocValues != null && histoDocValues.advanceExact(docId); + boolean isPresent = valueCounts != null && valueCounts.advanceExact(docId); currentDocId = isPresent ? docId : -1; return isPresent; } @@ -834,10 +899,10 @@ public ExponentialHistogram histogramValue() throws IOException { if (currentDocId == -1) { throw new IllegalStateException("No histogram present for current document"); } + boolean histoPresent = histoDocValues.advanceExact(currentDocId); boolean zeroThresholdPresent = zeroThresholds.advanceExact(currentDocId); - boolean valueCountsPresent = valueCounts.advanceExact(currentDocId); boolean valueSumsPresent = valueSums.advanceExact(currentDocId); - assert zeroThresholdPresent && valueCountsPresent && valueSumsPresent; + assert zeroThresholdPresent && histoPresent && valueSumsPresent; BytesRef encodedHistogram = histoDocValues.binaryValue(); double zeroThreshold = NumericUtils.sortableLongToDouble(zeroThresholds.longValue()); @@ -858,6 +923,11 @@ public ExponentialHistogram histogramValue() throws IOException { tempHistogram.reset(zeroThreshold, valueCount, valueSum, valueMin, valueMax, encodedHistogram); return tempHistogram; } + + @Override + public long valuesCountValue() throws IOException { + return valueCounts.longValue(); + } } private class ExponentialHistogramSyntheticFieldLoader implements CompositeSyntheticFieldLoader.DocValuesLayer { diff --git a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramMapperPlugin.java b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramMapperPlugin.java index 9f24b1cc4782d..b605185887abc 100644 --- a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramMapperPlugin.java +++ b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/ExponentialHistogramMapperPlugin.java @@ -10,15 +10,20 @@ import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.plugins.MapperPlugin; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; +import org.elasticsearch.xpack.exponentialhistogram.aggregations.metrics.ExponentialHistogramAggregatorsRegistrar; import java.util.Collections; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; +import java.util.function.Consumer; /** * Plugin adding support for exponential histogram field types. */ -public class ExponentialHistogramMapperPlugin extends Plugin implements MapperPlugin { +public class ExponentialHistogramMapperPlugin extends Plugin implements MapperPlugin, SearchPlugin { @Override public Map getMappers() { Map mappers = new LinkedHashMap<>(); @@ -27,4 +32,12 @@ public Map getMappers() { } return Collections.unmodifiableMap(mappers); } + + @Override + public List> getAggregationExtentions() { + if (ExponentialHistogramFieldMapper.EXPONENTIAL_HISTOGRAM_FEATURE.isEnabled()) { + return List.of(ExponentialHistogramAggregatorsRegistrar::registerValueCountAggregator); + } + return Collections.emptyList(); + } } diff --git a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/IndexWithCount.java b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/IndexWithCount.java index 7d9963e1e0c70..f74dcdadde81f 100644 --- a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/IndexWithCount.java +++ b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/IndexWithCount.java @@ -10,6 +10,7 @@ import org.elasticsearch.exponentialhistogram.CopyableBucketIterator; import org.elasticsearch.exponentialhistogram.ExponentialHistogram; +import java.util.ArrayList; import java.util.List; import java.util.OptionalLong; @@ -42,6 +43,15 @@ public long valueCount() { }; } + public static List fromIterator(CopyableBucketIterator iterator) { + List result = new ArrayList<>(); + while (iterator.hasNext()) { + result.add(new IndexWithCount(iterator.peekIndex(), iterator.peekCount())); + iterator.advance(); + } + return result; + } + private static class Iterator implements CopyableBucketIterator { private final List buckets; private final int scale; diff --git a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramAggregatorsRegistrar.java b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramAggregatorsRegistrar.java new file mode 100644 index 0000000000000..3416469cf39c3 --- /dev/null +++ b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramAggregatorsRegistrar.java @@ -0,0 +1,26 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.exponentialhistogram.aggregations.metrics; + +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceRegistry; +import org.elasticsearch.xpack.exponentialhistogram.aggregations.support.ExponentialHistogramValuesSourceType; + +/** + * Utility class providing static methods to register aggregators for the aggregate_metric values source + */ +public class ExponentialHistogramAggregatorsRegistrar { + + public static void registerValueCountAggregator(ValuesSourceRegistry.Builder builder) { + builder.register( + ValueCountAggregationBuilder.REGISTRY_KEY, + ExponentialHistogramValuesSourceType.EXPONENTIAL_HISTOGRAM, + ExponentialHistogramValueCountAggregator::new, + true + ); + } +} diff --git a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramValueCountAggregator.java b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramValueCountAggregator.java new file mode 100644 index 0000000000000..ed8cb1b3642f2 --- /dev/null +++ b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramValueCountAggregator.java @@ -0,0 +1,86 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.exponentialhistogram.aggregations.metrics; + +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.aggregations.AggregationExecutionContext; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.metrics.InternalValueCount; +import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregator; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.xpack.exponentialhistogram.aggregations.support.ExponentialHistogramValuesSource; +import org.elasticsearch.xpack.exponentialhistogram.fielddata.ExponentialHistogramValuesReader; + +import java.io.IOException; +import java.util.Map; + +class ExponentialHistogramValueCountAggregator extends NumericMetricsAggregator.SingleValue { + + private final ExponentialHistogramValuesSource.ExponentialHistogram valuesSource; + + // a count per bucket + private LongArray counts; + + ExponentialHistogramValueCountAggregator( + String name, + ValuesSourceConfig valuesSourceConfig, + AggregationContext aggregationContext, + Aggregator parent, + Map metadata + ) throws IOException { + super(name, aggregationContext, parent, metadata); + assert valuesSourceConfig.hasValues(); + this.valuesSource = (ExponentialHistogramValuesSource.ExponentialHistogram) valuesSourceConfig.getValuesSource(); + counts = bigArrays().newLongArray(1, true); + } + + @Override + public LeafBucketCollector getLeafCollector(AggregationExecutionContext aggCtx, final LeafBucketCollector sub) throws IOException { + BigArrays bigArrays = bigArrays(); + ExponentialHistogramValuesReader values = valuesSource.getHistogramValues(aggCtx.getLeafReaderContext()); + + return new LeafBucketCollectorBase(sub, values) { + @Override + public void collect(int doc, long bucket) throws IOException { + counts = bigArrays.grow(counts, bucket + 1); + if (values.advanceExact(doc)) { + counts.increment(bucket, values.valuesCountValue()); + } + } + }; + } + + @Override + public double metric(long owningBucketOrd) { + return owningBucketOrd >= counts.size() ? 0 : counts.get(owningBucketOrd); + } + + @Override + public InternalAggregation buildAggregation(long bucket) { + if (bucket >= counts.size()) { + return buildEmptyAggregation(); + } + return new InternalValueCount(name, counts.get(bucket), metadata()); + } + + @Override + public InternalAggregation buildEmptyAggregation() { + return InternalValueCount.empty(name, metadata()); + } + + @Override + public void doClose() { + Releasables.close(counts); + } + +} diff --git a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/fielddata/ExponentialHistogramValuesReader.java b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/fielddata/ExponentialHistogramValuesReader.java index 0006e0ab4efb1..927c0fb63b971 100644 --- a/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/fielddata/ExponentialHistogramValuesReader.java +++ b/x-pack/plugin/mapper-exponential-histogram/src/main/java/org/elasticsearch/xpack/exponentialhistogram/fielddata/ExponentialHistogramValuesReader.java @@ -28,5 +28,14 @@ public interface ExponentialHistogramValuesReader { */ ExponentialHistogram histogramValue() throws IOException; - // TODO: add accessors for min/max/sum/count which don't load the entire histogram + /** + * A shortcut for invoking {@link ExponentialHistogram#valueCount()} on the return value of {@link #histogramValue()}. + * This method is more performant because it avoids loading the unnecessary parts of the histogram. + * Must be called only after a successful call to {@link #advanceExact(int)}. + * + * @return the count of values in the histogram for the current document + */ + long valuesCountValue() throws IOException; + + // TODO: add accessors for min/max/sum which don't load the entire histogram } diff --git a/x-pack/plugin/mapper-exponential-histogram/src/test/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramAggregatorTestCase.java b/x-pack/plugin/mapper-exponential-histogram/src/test/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramAggregatorTestCase.java new file mode 100644 index 0000000000000..ecd7f2ff332f8 --- /dev/null +++ b/x-pack/plugin/mapper-exponential-histogram/src/test/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramAggregatorTestCase.java @@ -0,0 +1,89 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.exponentialhistogram.aggregations.metrics; + +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.exponentialhistogram.ExponentialHistogram; +import org.elasticsearch.exponentialhistogram.ExponentialHistogramBuilder; +import org.elasticsearch.exponentialhistogram.ExponentialHistogramCircuitBreaker; +import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.xpack.exponentialhistogram.ExponentialHistogramFieldMapper; +import org.elasticsearch.xpack.exponentialhistogram.ExponentialHistogramMapperPlugin; +import org.elasticsearch.xpack.exponentialhistogram.IndexWithCount; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +public abstract class ExponentialHistogramAggregatorTestCase extends AggregatorTestCase { + + @Override + protected List getSearchPlugins() { + return List.of(new ExponentialHistogramMapperPlugin()); + } + + protected static List createRandomHistograms(int count) { + return IntStream.range(0, count).mapToObj(i -> { + boolean hasNegativeValues = randomBoolean(); + boolean hasPositiveValues = randomBoolean(); + boolean hasZeroValues = randomBoolean(); + double[] rawValues = IntStream.concat( + IntStream.concat( + hasNegativeValues ? IntStream.range(0, randomIntBetween(1, 1000)).map(i1 -> -1) : IntStream.empty(), + hasPositiveValues ? IntStream.range(0, randomIntBetween(1, 1000)).map(i1 -> 1) : IntStream.empty() + ), + hasZeroValues ? IntStream.range(0, randomIntBetween(1, 100)).map(i1 -> 0) : IntStream.empty() + ).mapToDouble(sign -> sign * (Math.pow(1_000_000, randomDouble()))).toArray(); + + int numBuckets = randomIntBetween(4, 300); + ExponentialHistogram histo = ExponentialHistogram.create(numBuckets, ExponentialHistogramCircuitBreaker.noop(), rawValues); + // Randomize sum, min and max a little + if (histo.valueCount() > 0) { + ExponentialHistogramBuilder builder = ExponentialHistogram.builder(histo, ExponentialHistogramCircuitBreaker.noop()); + builder.sum(histo.sum() + (randomDouble() - 0.5) * 10_000); + builder.max(histo.max() - randomDouble()); + builder.min(histo.min() + randomDouble()); + histo = builder.build(); + } + return histo; + }).toList(); + } + + protected static void addHistogramDoc( + RandomIndexWriter iw, + String fieldName, + ExponentialHistogram histogram, + IndexableField... additionalFields + ) { + try { + ExponentialHistogramFieldMapper.HistogramDocValueFields docValues = ExponentialHistogramFieldMapper.buildDocValueFields( + fieldName, + histogram.scale(), + IndexWithCount.fromIterator(histogram.negativeBuckets().iterator()), + IndexWithCount.fromIterator(histogram.positiveBuckets().iterator()), + histogram.zeroBucket().zeroThreshold(), + histogram.valueCount(), + histogram.sum(), + nanToNull(histogram.min()), + nanToNull(histogram.max()) + ); + iw.addDocument(Stream.concat(docValues.fieldsAsList().stream(), Arrays.stream(additionalFields)).toList()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static Double nanToNull(double value) { + return Double.isNaN(value) ? null : value; + } + +} diff --git a/x-pack/plugin/mapper-exponential-histogram/src/test/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramValueCountAggregatorTests.java b/x-pack/plugin/mapper-exponential-histogram/src/test/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramValueCountAggregatorTests.java new file mode 100644 index 0000000000000..22ebc6cb690bc --- /dev/null +++ b/x-pack/plugin/mapper-exponential-histogram/src/test/java/org/elasticsearch/xpack/exponentialhistogram/aggregations/metrics/ExponentialHistogramValueCountAggregatorTests.java @@ -0,0 +1,121 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ +package org.elasticsearch.xpack.exponentialhistogram.aggregations.metrics; + +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.exponentialhistogram.ExponentialHistogram; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.InternalValueCount; +import org.elasticsearch.search.aggregations.metrics.ValueCountAggregationBuilder; +import org.elasticsearch.search.aggregations.support.AggregationInspectionHelper; +import org.elasticsearch.search.aggregations.support.CoreValuesSourceType; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.xpack.exponentialhistogram.ExponentialHistogramFieldMapper; +import org.elasticsearch.xpack.exponentialhistogram.aggregations.support.ExponentialHistogramValuesSourceType; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class ExponentialHistogramValueCountAggregatorTests extends ExponentialHistogramAggregatorTestCase { + + private static final String FIELD_NAME = "my_histogram"; + + public void testMatchesNumericDocValues() throws IOException { + + List histograms = createRandomHistograms(randomIntBetween(1, 1000)); + + long expectedCount = histograms.stream().mapToLong(ExponentialHistogram::valueCount).sum(); + + testCase(new MatchAllDocsQuery(), iw -> histograms.forEach(histo -> addHistogramDoc(iw, FIELD_NAME, histo)), valueCount -> { + assertThat(valueCount.getValue(), equalTo(expectedCount)); + assertThat(AggregationInspectionHelper.hasValue(valueCount), equalTo(expectedCount > 0)); + }); + } + + public void testNoDocs() throws IOException { + testCase(new MatchAllDocsQuery(), iw -> { + // Intentionally not writing any docs + }, valueCount -> { + assertThat(valueCount.getValue(), equalTo(0L)); + assertThat(AggregationInspectionHelper.hasValue(valueCount), equalTo(false)); + }); + } + + public void testNoMatchingField() throws IOException { + List histograms = createRandomHistograms(10); + testCase(new MatchAllDocsQuery(), iw -> histograms.forEach(histo -> addHistogramDoc(iw, "wrong_field", histo)), sum -> { + assertThat(sum.getValue(), equalTo(0L)); + assertThat(AggregationInspectionHelper.hasValue(sum), equalTo(false)); + }); + } + + public void testQueryFiltering() throws IOException { + List> histogramsWithFilter = createRandomHistograms(10).stream() + .map(histo -> Map.entry(histo, randomBoolean())) + .toList(); + + long filteredCount = histogramsWithFilter.stream() + .filter(Map.Entry::getValue) + .mapToLong(entry -> entry.getKey().valueCount()) + .sum(); + + testCase( + new TermQuery(new Term("match", "yes")), + iw -> histogramsWithFilter.forEach( + entry -> addHistogramDoc( + iw, + FIELD_NAME, + entry.getKey(), + new StringField("match", entry.getValue() ? "yes" : "no", Field.Store.NO) + ) + ), + sum -> { + assertThat(sum.getValue(), equalTo(filteredCount)); + assertThat(AggregationInspectionHelper.hasValue(sum), equalTo(filteredCount > 0)); + } + ); + } + + private void testCase(Query query, CheckedConsumer buildIndex, Consumer verify) + throws IOException { + var fieldType = new ExponentialHistogramFieldMapper.ExponentialHistogramFieldType(FIELD_NAME, Collections.emptyMap()); + AggregationBuilder aggregationBuilder = createAggBuilderForTypeTest(fieldType, FIELD_NAME); + testCase(buildIndex, verify, new AggTestConfig(aggregationBuilder, fieldType).withQuery(query)); + } + + @Override + protected AggregationBuilder createAggBuilderForTypeTest(MappedFieldType fieldType, String fieldName) { + return new ValueCountAggregationBuilder("value_count_agg").field(fieldName); + } + + @Override + protected List getSupportedValuesSourceTypes() { + return List.of( + CoreValuesSourceType.NUMERIC, + CoreValuesSourceType.KEYWORD, + CoreValuesSourceType.GEOPOINT, + CoreValuesSourceType.RANGE, + CoreValuesSourceType.BOOLEAN, + CoreValuesSourceType.DATE, + CoreValuesSourceType.IP, + ExponentialHistogramValuesSourceType.EXPONENTIAL_HISTOGRAM + ); + } +} diff --git a/x-pack/plugin/mapper-exponential-histogram/src/yamlRestTest/resources/rest-api-spec/test/20_aggregations.yml b/x-pack/plugin/mapper-exponential-histogram/src/yamlRestTest/resources/rest-api-spec/test/20_aggregations.yml new file mode 100644 index 0000000000000..eea5823f68f66 --- /dev/null +++ b/x-pack/plugin/mapper-exponential-histogram/src/yamlRestTest/resources/rest-api-spec/test/20_aggregations.yml @@ -0,0 +1,43 @@ +setup: + +# - skip: +# version: " - 8.12.99" +# reason: "exponential_histogram was added in 8.13" + + - do: + indices.create: + index: test_exponential_histogram + body: + mappings: + properties: + histo: + type: exponential_histogram + - do: + bulk: + index: test_exponential_histogram + refresh: true + body: + - '{"index": {}}' + - '{"histo":{"scale":-1,"sum":292.0,"min":-17.0,"max":101.0,"zero":{"count":3},"positive":{"indices":[-1,0,1,2,3],"counts":[1,3,5,3,2]},"negative":{"indices":[0,2],"counts":[1,1]}}}' + - '{"index": {}}' + - '{"histo":{"scale":38,"sum":0.0}}' + - '{"index": {}}' + - '{"histo":{"scale":38,"sum":0.0,"min":0.0,"max":0.0,"zero":{"count":5}}}' + - '{"index": {}}' + - '{"histo":{"scale":5,"sum":-4101.0,"min":-1000.0,"max":-700.0,"negative":{"indices":[302,308,314,318],"counts":[2,1,1,1]}}}' + - '{"index": {}}' + - '{"histo":{"scale":0,"sum":90000.0,"min":2000.0,"max":13000.0,"positive":{"indices":[10,11,12,13],"counts":[1,2,4,5]}}}' +--- +"Value count aggregation": + - do: + search: + index: test_exponential_histogram + size: 0 + body: + aggs: + value_count_agg: + value_count: + field: histo + + - match: { hits.total.value: 5 } + - match: { aggregations.value_count_agg.value: 41 }