Browse Source

Add unit tests for terms aggregation objects. (#23149)

Relates #22278
Adrien Grand 8 years ago
parent
commit
3bd1d46fc7

+ 11 - 0
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTerms.java

@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * Result of the {@link TermsAggregator} when the field is some kind of decimal number like a float, double, or distance.
@@ -99,6 +100,16 @@ public class DoubleTerms extends InternalMappedTerms<DoubleTerms, DoubleTerms.Bu
             builder.endObject();
             return builder;
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            return super.equals(obj) && Objects.equals(term, ((Bucket) obj).term);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), term);
+        }
     }
 
     public DoubleTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,

+ 17 - 0
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalMappedTerms.java

@@ -27,6 +27,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -110,4 +111,20 @@ public abstract class InternalMappedTerms<A extends InternalTerms<A, B>, B exten
         }
         return bucketMap.get(term);
     }
+
+    @Override
+    protected boolean doEquals(Object obj) {
+        InternalMappedTerms<?,?> that = (InternalMappedTerms<?,?>) obj;
+        return super.doEquals(obj)
+                && Objects.equals(buckets, that.buckets)
+                && Objects.equals(format, that.format)
+                && Objects.equals(otherDocCount, that.otherDocCount)
+                && Objects.equals(showTermDocCountError, that.showTermDocCountError)
+                && Objects.equals(shardSize, that.shardSize);
+    }
+
+    @Override
+    protected int doHashCode() {
+        return Objects.hash(super.doHashCode(), buckets, format, otherDocCount, showTermDocCountError, shardSize);
+    }
 }

+ 33 - 0
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTerms.java

@@ -36,6 +36,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 import static java.util.Collections.unmodifiableList;
 
@@ -135,6 +136,25 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
             InternalAggregations aggs = InternalAggregations.reduce(aggregationsList, context);
             return newBucket(docCount, aggs, docCountError);
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == null || getClass() != obj.getClass()) {
+                return false;
+            }
+            Bucket<?> that = (Bucket<?>) obj;
+            // No need to take format and showDocCountError, they are attributes
+            // of the parent terms aggregation object that are only copied here
+            // for serialization purposes
+            return Objects.equals(docCount, that.docCount)
+                    && Objects.equals(docCountError, that.docCountError)
+                    && Objects.equals(aggregations, that.aggregations);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(getClass(), docCount, docCountError, aggregations);
+        }
     }
 
     protected final Terms.Order order;
@@ -269,4 +289,17 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
      * Create an array to hold some buckets. Used in collecting the results.
      */
     protected abstract B[] createBucketsArray(int size);
+
+    @Override
+    protected boolean doEquals(Object obj) {
+        InternalTerms<?,?> that = (InternalTerms<?,?>) obj;
+        return Objects.equals(minDocCount, that.minDocCount)
+                && Objects.equals(order, that.order)
+                && Objects.equals(requiredSize, that.requiredSize);
+    }
+
+    @Override
+    protected int doHashCode() {
+        return Objects.hash(minDocCount, order, requiredSize);
+    }
 }

+ 11 - 0
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/LongTerms.java

@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * Result of the {@link TermsAggregator} when the field is some kind of whole number like a integer, long, or a date.
@@ -99,6 +100,16 @@ public class LongTerms extends InternalMappedTerms<LongTerms, LongTerms.Bucket>
             builder.endObject();
             return builder;
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            return super.equals(obj) && Objects.equals(term, ((Bucket) obj).term);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), term);
+        }
     }
 
     public LongTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,

+ 11 - 0
core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/StringTerms.java

@@ -29,6 +29,7 @@ import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * Result of the {@link TermsAggregator} when the field is a String.
@@ -95,6 +96,16 @@ public class StringTerms extends InternalMappedTerms<StringTerms, StringTerms.Bu
             builder.endObject();
             return builder;
         }
+
+        @Override
+        public boolean equals(Object obj) {
+            return super.equals(obj) && Objects.equals(termBytes, ((Bucket) obj).termBytes);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(super.hashCode(), termBytes);
+        }
     }
 
     public StringTerms(String name, Terms.Order order, int requiredSize, long minDocCount, List<PipelineAggregator> pipelineAggregators,

+ 25 - 2
core/src/test/java/org/elasticsearch/search/aggregations/InternalAggregationTestCase.java

@@ -36,14 +36,24 @@ public abstract class InternalAggregationTestCase<T extends InternalAggregation>
     private final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(
             new SearchModule(Settings.EMPTY, false, emptyList()).getNamedWriteables());
 
-    protected abstract T createTestInstance(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData);
+    protected abstract T createTestInstance(String name,
+            List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData);
+
+    /** Return an instance on an unmapped field. */
+    protected T createUnmappedInstance(String name,
+            List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        // For most impls, we use the same instance in the unmapped case and in the mapped case
+        return createTestInstance(name, pipelineAggregators, metaData);
+    }
 
     public final void testReduceRandom() {
         List<T> inputs = new ArrayList<>();
         List<InternalAggregation> toReduce = new ArrayList<>();
         int toReduceSize = between(1, 200);
         for (int i = 0; i < toReduceSize; i++) {
-            T t = createTestInstance();
+            T t = randomBoolean() ? createUnmappedInstance() : createTestInstance();
             inputs.add(t);
             toReduce.add(t);
         }
@@ -67,6 +77,19 @@ public abstract class InternalAggregationTestCase<T extends InternalAggregation>
         return createTestInstance(name, pipelineAggregators, metaData);
     }
 
+    /** Return an instance on an unmapped field. */
+    protected final T createUnmappedInstance() {
+        String name = randomAsciiOfLength(5);
+        List<PipelineAggregator> pipelineAggregators = new ArrayList<>();
+        // TODO populate pipelineAggregators
+        Map<String, Object> metaData = new HashMap<>();
+        int metaDataCount = randomBoolean() ? 0 : between(1, 10);
+        while (metaData.size() < metaDataCount) {
+            metaData.put(randomAsciiOfLength(5), randomAsciiOfLength(5));
+        }
+        return createUnmappedInstance(name, pipelineAggregators, metaData);
+    }
+
     @Override
     protected NamedWriteableRegistry getNamedWriteableRegistry() {
         return namedWriteableRegistry;

+ 66 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/DoubleTermsTests.java

@@ -0,0 +1,66 @@
+/*
+ * 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.bucket.terms;
+
+import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class DoubleTermsTests extends InternalTermsTestCase {
+
+    @Override
+    protected InternalTerms<?, ?> createTestInstance(
+            String name,
+            List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        Terms.Order order = Terms.Order.count(false);
+        long minDocCount = 1;
+        int requiredSize = 3;
+        int shardSize = requiredSize + 2;
+        DocValueFormat format = DocValueFormat.RAW;
+        boolean showTermDocCountError = false;
+        long docCountError = -1;
+        long otherDocCount = 0;
+        List<DoubleTerms.Bucket> buckets = new ArrayList<>();
+        final int numBuckets = randomInt(shardSize);
+        Set<Double> terms = new HashSet<>();
+        for (int i = 0; i < numBuckets; ++i) {
+            double term = randomValueOtherThanMany(d -> terms.add(d) == false, random()::nextDouble);
+            int docCount = randomIntBetween(1, 100);
+            buckets.add(new DoubleTerms.Bucket(term, docCount, InternalAggregations.EMPTY,
+                    showTermDocCountError, docCountError, format));
+        }
+        return new DoubleTerms(name, order, requiredSize, minDocCount, pipelineAggregators,
+                metaData, format, shardSize, showTermDocCountError, otherDocCount, buckets, docCountError);
+    }
+
+    @Override
+    protected Reader<InternalTerms<?, ?>> instanceReader() {
+        return DoubleTerms::new;
+    }
+
+}

+ 82 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/InternalTermsTestCase.java

@@ -0,0 +1,82 @@
+/*
+ * 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.bucket.terms;
+
+import org.elasticsearch.search.aggregations.InternalAggregationTestCase;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+public abstract class InternalTermsTestCase extends InternalAggregationTestCase<InternalTerms<?,?>> {
+
+    @Override
+    protected InternalTerms<?, ?> createUnmappedInstance(
+            String name,
+            List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        InternalTerms<?, ?> testInstance = createTestInstance(name, pipelineAggregators, metaData);
+        return new UnmappedTerms(name, testInstance.order, testInstance.requiredSize, testInstance.minDocCount,
+                pipelineAggregators, metaData);
+    }
+
+    @Override
+    protected void assertReduced(InternalTerms<?, ?> reduced, List<InternalTerms<?, ?>> inputs) {
+        final int requiredSize = inputs.get(0).requiredSize;
+        Map<Object, Long> reducedCounts = toCounts(reduced.getBuckets().stream());
+        Map<Object, Long> totalCounts = toCounts(inputs.stream().map(Terms::getBuckets).flatMap(List::stream));
+
+        assertEquals(reducedCounts.size() == requiredSize,
+                totalCounts.size() >= requiredSize);
+
+        Map<Object, Long> expectedReducedCounts = new HashMap<>(totalCounts);
+        expectedReducedCounts.keySet().retainAll(reducedCounts.keySet());
+        assertEquals(expectedReducedCounts, reducedCounts);
+
+        final long minFinalcount = reduced.getBuckets().isEmpty()
+                ? -1
+                : reduced.getBuckets().get(reduced.getBuckets().size() - 1).getDocCount();
+        Map<Object, Long> evictedTerms = new HashMap<>(totalCounts);
+        evictedTerms.keySet().removeAll(reducedCounts.keySet());
+        Optional<Entry<Object, Long>> missingTerm = evictedTerms.entrySet().stream()
+                .filter(e -> e.getValue() > minFinalcount).findAny();
+        if (missingTerm.isPresent()) {
+            fail("Missed term: " + missingTerm + " from " + reducedCounts);
+        }
+
+        final long reducedTotalDocCount = reduced.getSumOfOtherDocCounts()
+                + reduced.getBuckets().stream().mapToLong(Terms.Bucket::getDocCount).sum();
+        final long expectedTotalDocCount = inputs.stream().map(Terms::getBuckets)
+                .flatMap(List::stream).mapToLong(Terms.Bucket::getDocCount).sum();
+        assertEquals(expectedTotalDocCount, reducedTotalDocCount);
+    }
+
+    private static Map<Object, Long> toCounts(Stream<? extends Terms.Bucket> buckets) {
+        return buckets.collect(Collectors.toMap(
+                Terms.Bucket::getKey,
+                Terms.Bucket::getDocCount,
+                Long::sum));
+    }
+}

+ 66 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/LongTermsTests.java

@@ -0,0 +1,66 @@
+/*
+ * 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.bucket.terms;
+
+import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class LongTermsTests extends InternalTermsTestCase {
+
+    @Override
+    protected InternalTerms<?, ?> createTestInstance(
+            String name,
+            List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        Terms.Order order = Terms.Order.count(false);
+        long minDocCount = 1;
+        int requiredSize = 3;
+        int shardSize = requiredSize + 2;
+        DocValueFormat format = DocValueFormat.RAW;
+        boolean showTermDocCountError = false;
+        long docCountError = -1;
+        long otherDocCount = 0;
+        List<LongTerms.Bucket> buckets = new ArrayList<>();
+        final int numBuckets = randomInt(shardSize);
+        Set<Long> terms = new HashSet<>();
+        for (int i = 0; i < numBuckets; ++i) {
+            long term = randomValueOtherThanMany(l -> terms.add(l) == false, random()::nextLong);
+            int docCount = randomIntBetween(1, 100);
+            buckets.add(new LongTerms.Bucket(term, docCount, InternalAggregations.EMPTY,
+                    showTermDocCountError, docCountError, format));
+        }
+        return new LongTerms(name, order, requiredSize, minDocCount, pipelineAggregators,
+                metaData, format, shardSize, showTermDocCountError, otherDocCount, buckets, docCountError);
+    }
+
+    @Override
+    protected Reader<InternalTerms<?, ?>> instanceReader() {
+        return LongTerms::new;
+    }
+
+}

+ 67 - 0
core/src/test/java/org/elasticsearch/search/aggregations/bucket/terms/StringTermsTests.java

@@ -0,0 +1,67 @@
+/*
+ * 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.bucket.terms;
+
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.common.io.stream.Writeable.Reader;
+import org.elasticsearch.search.DocValueFormat;
+import org.elasticsearch.search.aggregations.InternalAggregations;
+import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class StringTermsTests extends InternalTermsTestCase {
+
+    @Override
+    protected InternalTerms<?, ?> createTestInstance(
+            String name,
+            List<PipelineAggregator> pipelineAggregators,
+            Map<String, Object> metaData) {
+        Terms.Order order = Terms.Order.count(false);
+        long minDocCount = 1;
+        int requiredSize = 3;
+        int shardSize = requiredSize + 2;
+        DocValueFormat format = DocValueFormat.RAW;
+        boolean showTermDocCountError = false;
+        long docCountError = -1;
+        long otherDocCount = 0;
+        List<StringTerms.Bucket> buckets = new ArrayList<>();
+        final int numBuckets = randomInt(shardSize);
+        Set<BytesRef> terms = new HashSet<>();
+        for (int i = 0; i < numBuckets; ++i) {
+            BytesRef term = randomValueOtherThanMany(b -> terms.add(b) == false, () -> new BytesRef(randomAsciiOfLength(10)));
+            int docCount = randomIntBetween(1, 100);
+            buckets.add(new StringTerms.Bucket(term, docCount, InternalAggregations.EMPTY,
+                    showTermDocCountError, docCountError, format));
+        }
+        return new StringTerms(name, order, requiredSize, minDocCount, pipelineAggregators,
+                metaData, format, shardSize, showTermDocCountError, otherDocCount, buckets, docCountError);
+    }
+
+    @Override
+    protected Reader<InternalTerms<?, ?>> instanceReader() {
+        return StringTerms::new;
+    }
+
+}