Drop deprecated aggregator wrapper (backport of #58367) (#58448)

This drops the deprecated and now unused `asMultiBucketAggregator`. It
was too easy to use it to make inefficient `Aggregators`.

Relates to #56487
This commit is contained in:
Nik Everett 2020-06-25 09:31:19 -04:00 committed by GitHub
parent ebe1d9cdbe
commit 335505c4e1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 4 additions and 369 deletions

View File

@ -19,180 +19,15 @@
package org.elasticsearch.search.aggregations;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.search.Scorable;
import org.apache.lucene.search.ScoreMode;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
import java.io.IOException;
import java.util.Map;
import java.util.function.BiConsumer;
import static org.elasticsearch.search.aggregations.support.AggregationUsageService.OTHER_SUBTYPE;
public abstract class AggregatorFactory {
public static final class MultiBucketAggregatorWrapper extends Aggregator {
private final BigArrays bigArrays;
private final Aggregator parent;
private final AggregatorFactory factory;
private final Aggregator first;
ObjectArray<Aggregator> aggregators;
ObjectArray<LeafBucketCollector> collectors;
MultiBucketAggregatorWrapper(BigArrays bigArrays, SearchContext context,
Aggregator parent, AggregatorFactory factory, Aggregator first) {
this.bigArrays = bigArrays;
this.parent = parent;
this.factory = factory;
this.first = first;
context.addReleasable(this, Lifetime.PHASE);
aggregators = bigArrays.newObjectArray(1);
aggregators.set(0, first);
collectors = bigArrays.newObjectArray(1);
}
public Class<?> getWrappedClass() {
return first.getClass();
}
@Override
public String name() {
return first.name();
}
@Override
public SearchContext context() {
return first.context();
}
@Override
public Aggregator parent() {
return first.parent();
}
@Override
public ScoreMode scoreMode() {
return first.scoreMode();
}
@Override
public Aggregator subAggregator(String name) {
throw new UnsupportedOperationException();
}
@Override
public void preCollection() throws IOException {
for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) {
aggregator.preCollection();
}
}
}
@Override
public void postCollection() throws IOException {
for (long i = 0; i < aggregators.size(); ++i) {
final Aggregator aggregator = aggregators.get(i);
if (aggregator != null) {
aggregator.postCollection();
}
}
}
@Override
public LeafBucketCollector getLeafCollector(final LeafReaderContext ctx) {
for (long i = 0; i < collectors.size(); ++i) {
collectors.set(i, null);
}
return new LeafBucketCollector() {
Scorable scorer;
@Override
public void setScorer(Scorable scorer) throws IOException {
this.scorer = scorer;
}
@Override
public void collect(int doc, long bucket) throws IOException {
collectors = bigArrays.grow(collectors, bucket + 1);
LeafBucketCollector collector = collectors.get(bucket);
if (collector == null) {
aggregators = bigArrays.grow(aggregators, bucket + 1);
Aggregator aggregator = aggregators.get(bucket);
if (aggregator == null) {
aggregator = factory.create(context(), parent, true);
aggregator.preCollection();
aggregators.set(bucket, aggregator);
}
collector = aggregator.getLeafCollector(ctx);
if (scorer != null) {
// Passing a null scorer can cause unexpected NPE at a later time,
// which can't not be directly linked to the fact that a null scorer has been supplied.
collector.setScorer(scorer);
}
collectors.set(bucket, collector);
}
collector.collect(doc, 0);
}
};
}
@Override
public InternalAggregation[] buildAggregations(long[] owningBucketOrds) throws IOException {
InternalAggregation[] results = new InternalAggregation[owningBucketOrds.length];
for (int ordIdx = 0; ordIdx < owningBucketOrds.length; ordIdx++) {
if (owningBucketOrds[ordIdx] < aggregators.size()) {
Aggregator aggregator = aggregators.get(owningBucketOrds[ordIdx]);
if (aggregator != null) {
/*
* This is the same call as buildTopLevel but since
* this aggregator may not be the top level we don't
* call that method here. It'd be weird sounding. And
* it'd trip assertions. Both bad.
*/
results[ordIdx] = aggregator.buildAggregations(new long [] {0})[0];
} else {
results[ordIdx] = buildEmptyAggregation();
}
} else {
results[ordIdx] = buildEmptyAggregation();
}
}
return results;
}
@Override
public InternalAggregation buildEmptyAggregation() {
return first.buildEmptyAggregation();
}
@Override
public void close() {
Releasables.close(aggregators, collectors);
}
@Override
public void collectDebugInfo(BiConsumer<String, Object> add) {
/*
* There isn't really a sane way to give our delegates a way to
* add entries because we'd have to merge them. So we just *don't*
* and leave a marker of our own. This ain't great, but we plan
* to cut down on usage of this wrapper in the future.
*/
add.accept("wrapped_in_multi_bucket_aggregator", true);
super.collectDebugInfo(add);
}
}
protected final String name;
protected final AggregatorFactory parent;
protected final AggregatorFactories factories;
@ -254,20 +89,6 @@ public abstract class AggregatorFactory {
return parent;
}
/**
* Utility method. Given an {@link AggregatorFactory} that creates
* {@link Aggregator}s that only know how to collect bucket {@code 0}, this
* returns an aggregator that can collect any bucket.
* @deprecated implement the aggregator to handle many owning buckets
*/
@Deprecated
protected static Aggregator asMultiBucketAggregator(final AggregatorFactory factory, final SearchContext searchContext,
final Aggregator parent) throws IOException {
final Aggregator first = factory.create(searchContext, parent, true);
final BigArrays bigArrays = searchContext.bigArrays();
return new MultiBucketAggregatorWrapper(bigArrays, searchContext, parent, factory, first);
}
/**
* Returns the aggregation subtype for nodes usage stats.
* <p>

View File

@ -21,12 +21,10 @@ package org.elasticsearch.search.aggregations.bucket;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.IntArray;
import org.elasticsearch.common.util.LongHash;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorBase;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.LeafBucketCollector;
@ -154,7 +152,7 @@ public abstract class BucketsAggregator extends AggregatorBase {
* <p>
* Most aggregations should probably use something like
* {@link #buildSubAggsForAllBuckets(Object[][], ToLongFunction, BiConsumer)}
* or {@link #buildAggregationsForVariableBuckets(long[], LongHash, BucketBuilderForVariable, Function)}
* or {@link #buildAggregationsForVariableBuckets(long[], LongKeyedBucketOrds, BucketBuilderForVariable, ResultBuilderForVariable)}
* or {@link #buildAggregationsForFixedBucketCount(long[], int, BucketBuilderForFixedCount, Function)}
* or {@link #buildAggregationsForSingleBucket(long[], SingleBucketResultBuilder)}
* instead of calling this directly.
@ -300,32 +298,7 @@ public abstract class BucketsAggregator extends AggregatorBase {
/**
* Build aggregation results for an aggregator with a varying number of
* {@code long} keyed buckets that is at the top level or wrapped in
* {@link AggregatorFactory#asMultiBucketAggregator}.
* @param owningBucketOrds owning bucket ordinals for which to build the results
* @param bucketOrds hash of values to the bucket ordinal
*/
protected final <B> InternalAggregation[] buildAggregationsForVariableBuckets(long[] owningBucketOrds, LongHash bucketOrds,
BucketBuilderForVariable<B> bucketBuilder, Function<List<B>, InternalAggregation> resultBuilder) throws IOException {
assert owningBucketOrds.length == 1 && owningBucketOrds[0] == 0;
long[] bucketOrdsToCollect = new long[(int) bucketOrds.size()];
for (int bucketOrd = 0; bucketOrd < bucketOrds.size(); bucketOrd++) {
bucketOrdsToCollect[bucketOrd] = bucketOrd;
}
InternalAggregations[] subAggregationResults = buildSubAggsForBuckets(bucketOrdsToCollect);
List<B> buckets = new ArrayList<>((int) bucketOrds.size());
for (int bucketOrd = 0; bucketOrd < bucketOrds.size(); bucketOrd++) {
buckets.add(bucketBuilder.build(bucketOrds.get(bucketOrd), bucketDocCount(bucketOrd), subAggregationResults[bucketOrd]));
}
return new InternalAggregation[] { resultBuilder.apply(buckets) };
}
/**
* Build aggregation results for an aggregator with a varying number of
* {@code long} keyed buckets that is at the top level or wrapped in
* {@link AggregatorFactory#asMultiBucketAggregator}.
* {@code long} keyed buckets.
* @param owningBucketOrds owning bucket ordinals for which to build the results
* @param bucketOrds hash of values to the bucket ordinal
*/

View File

@ -20,7 +20,6 @@
package org.elasticsearch.search.profile.aggregation;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactory.MultiBucketAggregatorWrapper;
import org.elasticsearch.search.profile.AbstractInternalProfileTree;
public class InternalAggregationProfileTree extends AbstractInternalProfileTree<AggregationProfileBreakdown, Aggregator> {
@ -38,9 +37,6 @@ public class InternalAggregationProfileTree extends AbstractInternalProfileTree<
if (element.getClass().getSimpleName().isEmpty()) {
return element.getClass().getSuperclass().getSimpleName();
}
if (element instanceof MultiBucketAggregatorWrapper) {
return ((MultiBucketAggregatorWrapper) element).getWrappedClass().getSimpleName();
}
Class<?> enclosing = element.getClass().getEnclosingClass();
if (enclosing != null) {
return enclosing.getSimpleName() + "." + element.getClass().getSimpleName();

View File

@ -1,77 +0,0 @@
/*
* 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;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.memory.MemoryIndex;
import org.apache.lucene.search.Scorer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.common.util.MockPageCacheRecycler;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.ESTestCase;
import java.util.Collections;
import static org.mockito.Matchers.same;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.reset;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
public class MultiBucketAggregatorWrapperTests extends ESTestCase {
public void testNoNullScorerIsDelegated() throws Exception {
LeafReaderContext leafReaderContext = MemoryIndex.fromDocument(Collections.emptyList(), new MockAnalyzer(random()))
.createSearcher().getIndexReader().leaves().get(0);
BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService());
QueryShardContext queryShardContext = mock(QueryShardContext.class);
when(queryShardContext.bigArrays()).thenReturn(bigArrays);
SearchContext searchContext = mock(SearchContext.class);
when(searchContext.bigArrays()).thenReturn(bigArrays);
Aggregator aggregator = mock(Aggregator.class);
AggregatorFactory aggregatorFactory = new TestAggregatorFactory(queryShardContext, aggregator);
LeafBucketCollector wrappedCollector = mock(LeafBucketCollector.class);
when(aggregator.getLeafCollector(leafReaderContext)).thenReturn(wrappedCollector);
Aggregator wrapper = AggregatorFactory.asMultiBucketAggregator(aggregatorFactory, searchContext, null);
LeafBucketCollector collector = wrapper.getLeafCollector(leafReaderContext);
collector.collect(0, 0);
// setScorer should not be invoked as it has not been set
// Only collect should be invoked:
verify(wrappedCollector).collect(0, 0);
verifyNoMoreInteractions(wrappedCollector);
reset(wrappedCollector);
Scorer scorer = mock(Scorer.class);
collector.setScorer(scorer);
collector.collect(0, 1);
verify(wrappedCollector).setScorer(same(scorer));
verify(wrappedCollector).collect(0, 0);
verifyNoMoreInteractions(wrappedCollector);
wrapper.close();
}
}

View File

@ -1,64 +0,0 @@
/*
* 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;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.common.util.MockPageCacheRecycler;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
/**
* Test implementation for AggregatorFactory.
*/
public class TestAggregatorFactory extends AggregatorFactory {
private final Aggregator aggregator;
TestAggregatorFactory(QueryShardContext queryShardContext, Aggregator aggregator) throws IOException {
super("_name", queryShardContext, null, new AggregatorFactories.Builder(), Collections.emptyMap());
this.aggregator = aggregator;
}
@Override
protected Aggregator createInternal(SearchContext searchContext, Aggregator parent, boolean collectsFromSingleBucket,
Map metadata) throws IOException {
return aggregator;
}
public static TestAggregatorFactory createInstance() throws IOException {
BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService());
QueryShardContext queryShardContext = mock(QueryShardContext.class);
when(queryShardContext.bigArrays()).thenReturn(bigArrays);
Aggregator aggregator = mock(Aggregator.class);
return new TestAggregatorFactory(queryShardContext, aggregator);
}
}

View File

@ -57,7 +57,6 @@ import org.elasticsearch.script.ScriptModule;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.ScriptType;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.BucketOrder;
import org.elasticsearch.search.aggregations.InternalAggregation;
@ -65,7 +64,6 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter;
import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms;
import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
import org.elasticsearch.search.aggregations.bucket.terms.NumericTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
@ -778,12 +776,7 @@ public class NestedAggregatorTests extends AggregatorTestCase {
}
}
/**
* {@link NumericTermsAggregator} is the first complex bucking aggregation
* that stopped wrapping itself in {@link AggregatorFactory#asMultiBucketAggregator}
* so this tests that nested works properly inside of it.
*/
public void testNestedUnderLongTerms() throws IOException {
public void testNestedUnderTerms() throws IOException {
int numProducts = scaledRandomIntBetween(1, 100);
int numResellers = scaledRandomIntBetween(1, 100);

View File

@ -34,11 +34,9 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper;
import org.elasticsearch.index.mapper.TypeFieldMapper;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorTestCase;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.bucket.terms.LongTerms;
import org.elasticsearch.search.aggregations.bucket.terms.NumericTermsAggregator;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder;
import org.elasticsearch.search.aggregations.metrics.InternalMax;
import org.elasticsearch.search.aggregations.metrics.MaxAggregationBuilder;
@ -236,12 +234,7 @@ public class ReverseNestedAggregatorTests extends AggregatorTestCase {
}
}
/**
* {@link NumericTermsAggregator} is the first complex bucking aggregation
* that stopped wrapping itself in {@link AggregatorFactory#asMultiBucketAggregator}
* so this tests that nested works properly inside of it.
*/
public void testNestedUnderLongTerms() throws IOException {
public void testNestedUnderTerms() throws IOException {
int numProducts = scaledRandomIntBetween(1, 100);
int numResellers = scaledRandomIntBetween(1, 100);