瀏覽代碼

Add unit tests to percentile ranks aggregations. (#23240)

Relates #22278
Adrien Grand 8 年之前
父節點
當前提交
3134d6b520

+ 18 - 0
core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/AbstractInternalHDRPercentiles.java

@@ -30,8 +30,10 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.zip.DataFormatException;
 
 abstract class AbstractInternalHDRPercentiles extends InternalNumericMetricsAggregation.MultiValue {
@@ -139,4 +141,20 @@ abstract class AbstractInternalHDRPercentiles extends InternalNumericMetricsAggr
         }
         return builder;
     }
+
+    @Override
+    protected boolean doEquals(Object obj) {
+        AbstractInternalHDRPercentiles that = (AbstractInternalHDRPercentiles) obj;
+        return keyed == that.keyed
+                && Arrays.equals(keys, that.keys)
+                && Objects.equals(state, that.state);
+    }
+
+    @Override
+    protected int doHashCode() {
+        // we cannot use state.hashCode at the moment because of:
+        // https://github.com/HdrHistogram/HdrHistogram/issues/81
+        // TODO: upgrade the HDRHistogram library
+        return Objects.hash(keyed, Arrays.hashCode(keys), state.getIntegerToDoubleValueConversionRatio(), state.getTotalCount());
+    }
 }

+ 15 - 0
core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/AbstractInternalTDigestPercentiles.java

@@ -28,8 +28,10 @@ import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggre
 import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 abstract class AbstractInternalTDigestPercentiles extends InternalNumericMetricsAggregation.MultiValue {
 
@@ -122,4 +124,17 @@ abstract class AbstractInternalTDigestPercentiles extends InternalNumericMetrics
         }
         return builder;
     }
+
+    @Override
+    protected boolean doEquals(Object obj) {
+        AbstractInternalTDigestPercentiles that = (AbstractInternalTDigestPercentiles) obj;
+        return keyed == that.keyed
+                && Arrays.equals(keys, that.keys)
+                && Objects.equals(state, that.state);
+    }
+
+    @Override
+    protected int doHashCode() {
+        return Objects.hash(keyed, Arrays.hashCode(keys), state);
+    }
 }

+ 5 - 0
core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentileRanks.java

@@ -116,4 +116,9 @@ public class InternalTDigestPercentileRanks extends AbstractInternalTDigestPerce
             throw new UnsupportedOperationException();
         }
     }
+
+    @Override
+    protected boolean doEquals(Object obj) {
+        return super.doEquals(obj);
+    }
 }

+ 35 - 0
core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/TDigestState.java

@@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 
 import java.io.IOException;
+import java.util.Iterator;
 
 /**
  * Extension of {@link com.tdunning.math.stats.TDigest} with custom serialization.
@@ -61,4 +62,38 @@ public class TDigestState extends AVLTreeDigest {
         return state;
     }
 
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null || obj instanceof TDigestState == false) {
+            return false;
+        }
+        TDigestState that = (TDigestState) obj;
+        if (compression != that.compression) {
+            return false;
+        }
+        Iterator<? extends Centroid> thisCentroids = centroids().iterator();
+        Iterator<? extends Centroid> thatCentroids = centroids().iterator();
+        while (thisCentroids.hasNext()) {
+            if (thatCentroids.hasNext() == false) {
+                return false;
+            }
+            Centroid thisNext = thisCentroids.next();
+            Centroid thatNext = thatCentroids.next();
+            if (thisNext.mean() != thatNext.mean() || thisNext.count() != thatNext.count()) {
+                return false;
+            }
+        }
+        return thatCentroids.hasNext() == false;
+    }
+
+    @Override
+    public int hashCode() {
+        int h = getClass().hashCode();
+        h = 31 * h + Double.hashCode(compression);
+        for (Centroid centroid : centroids()) {
+            h = 31 * h + Double.hashCode(centroid.mean());
+            h = 31 * h + centroid.count();
+        }
+        return h;
+    }
 }

+ 94 - 0
core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/HDRPercentileRanksAggregatorTests.java

@@ -0,0 +1,94 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.percentiles.hdr;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.NumericUtils;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.mapper.NumberFieldMapper;
+import org.elasticsearch.search.aggregations.AggregatorTestCase;
+import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
+import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanks;
+import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksAggregationBuilder;
+import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
+import org.hamcrest.Matchers;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+public class HDRPercentileRanksAggregatorTests extends AggregatorTestCase {
+
+    public void testEmpty() throws IOException {
+        PercentileRanksAggregationBuilder aggBuilder = new PercentileRanksAggregationBuilder("my_agg")
+                .field("field")
+                .method(PercentilesMethod.HDR)
+                .values(0.5);
+        MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);
+        fieldType.setName("field");
+        try (IndexReader reader = new MultiReader()) {
+            IndexSearcher searcher = new IndexSearcher(reader);
+            PercentileRanks ranks = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
+            Percentile rank = ranks.iterator().next();
+            assertEquals(Double.NaN, rank.getPercent(), 0d);
+            assertEquals(0.5, rank.getValue(), 0d);
+        }
+    }
+
+    public void testSimple() throws IOException {
+        try (Directory dir = newDirectory();
+                RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
+            for (double value : new double[] {3, 0.2, 10}) {
+                Document doc = new Document();
+                doc.add(new SortedNumericDocValuesField("field", NumericUtils.doubleToSortableLong(value)));
+                w.addDocument(doc);
+            }
+
+            PercentileRanksAggregationBuilder aggBuilder = new PercentileRanksAggregationBuilder("my_agg")
+                    .field("field")
+                    .method(PercentilesMethod.HDR)
+                    .values(0.1, 0.5, 12);
+            MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);
+            fieldType.setName("field");
+            try (IndexReader reader = w.getReader()) {
+                IndexSearcher searcher = new IndexSearcher(reader);
+                PercentileRanks ranks = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
+                Iterator<Percentile> rankIterator = ranks.iterator();
+                Percentile rank = rankIterator.next();
+                assertEquals(0.1, rank.getValue(), 0d);
+                assertThat(rank.getPercent(), Matchers.equalTo(0d));
+                rank = rankIterator.next();
+                assertEquals(0.5, rank.getValue(), 0d);
+                assertThat(rank.getPercent(), Matchers.greaterThan(0d));
+                assertThat(rank.getPercent(), Matchers.lessThan(100d));
+                rank = rankIterator.next();
+                assertEquals(12, rank.getValue(), 0d);
+                assertThat(rank.getPercent(), Matchers.equalTo(100d));
+                assertFalse(rankIterator.hasNext());
+            }
+        }
+    }
+}

+ 64 - 0
core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/hdr/InternalHDRPercentilesRanksTests.java

@@ -0,0 +1,64 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.percentiles.hdr;
+
+import org.HdrHistogram.DoubleHistogram;
+import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
+import org.elasticsearch.search.aggregations.metrics.percentiles.hdr.InternalHDRPercentileRanks;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.util.List;
+import java.util.Map;
+
+public class InternalHDRPercentilesRanksTests extends InternalAggregationTestCase<InternalHDRPercentileRanks> {
+
+    @Override
+    protected InternalHDRPercentileRanks createTestInstance(String name, List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        double[] cdfValues = new double[] { 0.5 };
+        int numberOfSignificantValueDigits = 3;
+        DoubleHistogram state = new DoubleHistogram(numberOfSignificantValueDigits);
+        int numValues = randomInt(100);
+        for (int i = 0; i < numValues; ++i) {
+            state.recordValue(randomDouble());
+        }
+        boolean keyed = false;
+        DocValueFormat format = DocValueFormat.RAW;
+        return new InternalHDRPercentileRanks(name, cdfValues, state, keyed, format, pipelineAggregators, metaData);
+    }
+
+    @Override
+    protected void assertReduced(InternalHDRPercentileRanks reduced, List<InternalHDRPercentileRanks> inputs) {
+        // it is hard to check the values due to the inaccuracy of the algorithm
+        long totalCount = 0;
+        for (InternalHDRPercentileRanks ranks : inputs) {
+            totalCount += ranks.state.getTotalCount();
+        }
+        assertEquals(totalCount, reduced.state.getTotalCount());
+    }
+
+    @Override
+    protected Reader<InternalHDRPercentileRanks> instanceReader() {
+        return InternalHDRPercentileRanks::new;
+    }
+
+}

+ 74 - 0
core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/InternalTDigestPercentilesRanksTests.java

@@ -0,0 +1,74 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest;
+
+import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.util.List;
+import java.util.Map;
+
+public class InternalTDigestPercentilesRanksTests extends InternalAggregationTestCase<InternalTDigestPercentileRanks> {
+
+    @Override
+    protected InternalTDigestPercentileRanks createTestInstance(String name, List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        double[] cdfValues = new double[] { 0.5 };
+        TDigestState state = new TDigestState(100);
+        int numValues = randomInt(100);
+        for (int i = 0; i < numValues; ++i) {
+            state.add(randomDouble());
+        }
+        boolean keyed = false;
+        DocValueFormat format = DocValueFormat.RAW;
+        return new InternalTDigestPercentileRanks(name, cdfValues, state, keyed, format, pipelineAggregators, metaData);
+    }
+
+    @Override
+    protected void assertReduced(InternalTDigestPercentileRanks reduced, List<InternalTDigestPercentileRanks> inputs) {
+        // it is hard to check the values due to the inaccuracy of the algorithm
+        // the min/max values should be accurate due to the way the algo works so we can at least test those
+        double min = Double.POSITIVE_INFINITY;
+        double max = Double.NEGATIVE_INFINITY;
+        long totalCount = 0;
+        for (InternalTDigestPercentileRanks ranks : inputs) {
+            if (ranks.state.centroidCount() == 0) {
+                // quantiles would return NaN
+                continue;
+            }
+            totalCount += ranks.state.size();
+            min = Math.min(ranks.state.quantile(0), min);
+            max = Math.max(ranks.state.quantile(1), max);
+        }
+        assertEquals(totalCount, reduced.state.size());
+        if (totalCount > 0) {
+            assertEquals(reduced.state.quantile(0), min, 0d);
+            assertEquals(reduced.state.quantile(1), max, 0d);
+        }
+    }
+
+    @Override
+    protected Reader<InternalTDigestPercentileRanks> instanceReader() {
+        return InternalTDigestPercentileRanks::new;
+    }
+
+}

+ 98 - 0
core/src/test/java/org/elasticsearch/search/aggregations/metrics/percentiles/tdigest/TDigestPercentileRanksAggregatorTests.java

@@ -0,0 +1,98 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.search.aggregations.metrics.percentiles.tdigest;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.SortedNumericDocValuesField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.NumericUtils;
+import org.elasticsearch.index.mapper.MappedFieldType;
+import org.elasticsearch.index.mapper.NumberFieldMapper;
+import org.elasticsearch.search.aggregations.AggregatorTestCase;
+import org.elasticsearch.search.aggregations.metrics.percentiles.Percentile;
+import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanks;
+import org.elasticsearch.search.aggregations.metrics.percentiles.PercentileRanksAggregationBuilder;
+import org.elasticsearch.search.aggregations.metrics.percentiles.PercentilesMethod;
+import org.hamcrest.Matchers;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+public class TDigestPercentileRanksAggregatorTests extends AggregatorTestCase {
+
+    public void testEmpty() throws IOException {
+        PercentileRanksAggregationBuilder aggBuilder = new PercentileRanksAggregationBuilder("my_agg")
+                .field("field")
+                .method(PercentilesMethod.TDIGEST)
+                .values(0.5);
+        MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);
+        fieldType.setName("field");
+        try (IndexReader reader = new MultiReader()) {
+            IndexSearcher searcher = new IndexSearcher(reader);
+            PercentileRanks ranks = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
+            Percentile rank = ranks.iterator().next();
+            assertEquals(Double.NaN, rank.getPercent(), 0d);
+            assertEquals(0.5, rank.getValue(), 0d);
+        }
+    }
+
+    public void testSimple() throws IOException {
+        try (Directory dir = newDirectory();
+                RandomIndexWriter w = new RandomIndexWriter(random(), dir)) {
+            for (double value : new double[] {3, 0.2, 10}) {
+                Document doc = new Document();
+                doc.add(new SortedNumericDocValuesField("field", NumericUtils.doubleToSortableLong(value)));
+                w.addDocument(doc);
+            }
+
+            PercentileRanksAggregationBuilder aggBuilder = new PercentileRanksAggregationBuilder("my_agg")
+                    .field("field")
+                    .method(PercentilesMethod.TDIGEST)
+                    .values(0.1, 0.5, 12);
+            MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.DOUBLE);
+            fieldType.setName("field");
+            try (IndexReader reader = w.getReader()) {
+                IndexSearcher searcher = new IndexSearcher(reader);
+                PercentileRanks ranks = search(searcher, new MatchAllDocsQuery(), aggBuilder, fieldType);
+                Iterator<Percentile> rankIterator = ranks.iterator();
+                Percentile rank = rankIterator.next();
+                assertEquals(0.1, rank.getValue(), 0d);
+                // TODO: Fix T-Digest: this assertion should pass but we currently get ~15
+                // https://github.com/elastic/elasticsearch/issues/14851
+                // assertThat(rank.getPercent(), Matchers.equalTo(0d));
+                rank = rankIterator.next();
+                assertEquals(0.5, rank.getValue(), 0d);
+                assertThat(rank.getPercent(), Matchers.greaterThan(0d));
+                assertThat(rank.getPercent(), Matchers.lessThan(100d));
+                rank = rankIterator.next();
+                assertEquals(12, rank.getValue(), 0d);
+                // TODO: Fix T-Digest: this assertion should pass but we currently get ~59
+                // https://github.com/elastic/elasticsearch/issues/14851
+                // assertThat(rank.getPercent(), Matchers.equalTo(100d));
+                assertFalse(rankIterator.hasNext());
+            }
+        }
+    }
+}

+ 0 - 3
test/framework/src/main/java/org/elasticsearch/test/AbstractWireSerializingTestCase.java

@@ -61,10 +61,7 @@ public abstract class AbstractWireSerializingTestCase<T extends Writeable> exten
 
             T secondInstance = copyInstance(firstInstance);
             assertEquals("instance is not equal to self", secondInstance, secondInstance);
-            if (false == firstInstance.equals(secondInstance)) {
-                firstInstance.equals(secondInstance);
             assertEquals("instance is not equal to its copy", firstInstance, secondInstance);
-            }
             assertEquals("equals is not symmetric", secondInstance, firstInstance);
             assertThat("instance copy's hashcode is different from original hashcode", secondInstance.hashCode(),
                     equalTo(firstInstance.hashCode()));