fixed issue with eggs in percolation request for 1 shard

This commit is contained in:
Colin Goodheart-Smithe 2015-04-29 14:54:05 +01:00
parent ccca0386ef
commit 3bb8ff2a92
2 changed files with 78 additions and 21 deletions

View File

@ -847,16 +847,11 @@ public class PercolatorService extends AbstractComponent {
return null; return null;
} }
InternalAggregations aggregations;
if (shardResults.size() == 1) {
aggregations = shardResults.get(0).aggregations();
} else {
List<InternalAggregations> aggregationsList = new ArrayList<>(shardResults.size()); List<InternalAggregations> aggregationsList = new ArrayList<>(shardResults.size());
for (PercolateShardResponse shardResult : shardResults) { for (PercolateShardResponse shardResult : shardResults) {
aggregationsList.add(shardResult.aggregations()); aggregationsList.add(shardResult.aggregations());
} }
aggregations = InternalAggregations.reduce(aggregationsList, new ReduceContext(bigArrays, scriptService)); InternalAggregations aggregations = InternalAggregations.reduce(aggregationsList, new ReduceContext(bigArrays, scriptService));
}
if (aggregations != null) { if (aggregations != null) {
List<SiblingReducer> reducers = shardResults.get(0).reducers(); List<SiblingReducer> reducers = shardResults.get(0).reducers();
if (reducers != null) { if (reducers != null) {

View File

@ -20,12 +20,14 @@ package org.elasticsearch.percolator;
import org.elasticsearch.action.percolate.PercolateRequestBuilder; import org.elasticsearch.action.percolate.PercolateRequestBuilder;
import org.elasticsearch.action.percolate.PercolateResponse; import org.elasticsearch.action.percolate.PercolateResponse;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.Aggregation; import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.Aggregations;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.bucket.terms.Terms; import org.elasticsearch.search.aggregations.bucket.terms.Terms;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
import org.elasticsearch.search.aggregations.reducers.ReducerBuilders; import org.elasticsearch.search.aggregations.reducers.ReducerBuilders;
import org.elasticsearch.search.aggregations.reducers.bucketmetrics.InternalBucketMetricValue; import org.elasticsearch.search.aggregations.reducers.bucketmetrics.InternalBucketMetricValue;
import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.ElasticsearchIntegrationTest;
@ -70,20 +72,18 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati
expectedCount[i % numUniqueQueries]++; expectedCount[i % numUniqueQueries]++;
QueryBuilder queryBuilder = matchQuery("field1", value); QueryBuilder queryBuilder = matchQuery("field1", value);
client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i)) client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i))
.setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()) .setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()).execute()
.execute().actionGet(); .actionGet();
} }
client().admin().indices().prepareRefresh("test").execute().actionGet(); client().admin().indices().prepareRefresh("test").execute().actionGet();
for (int i = 0; i < numQueries; i++) { for (int i = 0; i < numQueries; i++) {
String value = values[i % numUniqueQueries]; String value = values[i % numUniqueQueries];
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate() PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type")
.setIndices("test").setDocumentType("type")
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject())); .setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()));
SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values()); SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2") percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2").collectMode(aggCollectionMode));
.collectMode(aggCollectionMode ));
if (randomBoolean()) { if (randomBoolean()) {
percolateRequestBuilder.setPercolateQuery(matchAllQuery()); percolateRequestBuilder.setPercolateQuery(matchAllQuery());
@ -135,20 +135,18 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati
expectedCount[i % numUniqueQueries]++; expectedCount[i % numUniqueQueries]++;
QueryBuilder queryBuilder = matchQuery("field1", value); QueryBuilder queryBuilder = matchQuery("field1", value);
client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i)) client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i))
.setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()) .setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", "b").endObject()).execute()
.execute().actionGet(); .actionGet();
} }
client().admin().indices().prepareRefresh("test").execute().actionGet(); client().admin().indices().prepareRefresh("test").execute().actionGet();
for (int i = 0; i < numQueries; i++) { for (int i = 0; i < numQueries; i++) {
String value = values[i % numUniqueQueries]; String value = values[i % numUniqueQueries];
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate() PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type")
.setIndices("test").setDocumentType("type")
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject())); .setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()));
SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values()); SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2") percolateRequestBuilder.addAggregation(AggregationBuilders.terms("a").field("field2").collectMode(aggCollectionMode));
.collectMode(aggCollectionMode ));
if (randomBoolean()) { if (randomBoolean()) {
percolateRequestBuilder.setPercolateQuery(matchAllQuery()); percolateRequestBuilder.setPercolateQuery(matchAllQuery());
@ -186,7 +184,7 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati
assertThat(maxA, notNullValue()); assertThat(maxA, notNullValue());
assertThat(maxA.getName(), equalTo("max_a")); assertThat(maxA.getName(), equalTo("max_a"));
assertThat(maxA.value(), equalTo((double) expectedCount[i % values.length])); assertThat(maxA.value(), equalTo((double) expectedCount[i % values.length]));
assertThat(maxA.keys(), equalTo(new String[] {"b"})); assertThat(maxA.keys(), equalTo(new String[] { "b" }));
} }
} }
@ -194,12 +192,76 @@ public class PercolatorFacetsAndAggregationsTests extends ElasticsearchIntegrati
public void testSignificantAggs() throws Exception { public void testSignificantAggs() throws Exception {
client().admin().indices().prepareCreate("test").execute().actionGet(); client().admin().indices().prepareCreate("test").execute().actionGet();
ensureGreen(); ensureGreen();
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate() PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type")
.setIndices("test").setDocumentType("type")
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", "value").endObject())) .setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", "value").endObject()))
.addAggregation(AggregationBuilders.significantTerms("a").field("field2")); .addAggregation(AggregationBuilders.significantTerms("a").field("field2"));
PercolateResponse response = percolateRequestBuilder.get(); PercolateResponse response = percolateRequestBuilder.get();
assertNoFailures(response); assertNoFailures(response);
} }
@Test
public void testSingleShardAggregations() throws Exception {
assertAcked(prepareCreate("test").setSettings(ImmutableSettings.builder().put(indexSettings()).put("SETTING_NUMBER_OF_SHARDS", 1))
.addMapping("type", "field1", "type=string", "field2", "type=string"));
ensureGreen();
int numQueries = scaledRandomIntBetween(250, 500);
logger.info("--> registering {} queries", numQueries);
for (int i = 0; i < numQueries; i++) {
String value = "value0";
QueryBuilder queryBuilder = matchQuery("field1", value);
client().prepareIndex("test", PercolatorService.TYPE_NAME, Integer.toString(i))
.setSource(jsonBuilder().startObject().field("query", queryBuilder).field("field2", i % 3 == 0 ? "b" : "a").endObject())
.execute()
.actionGet();
}
client().admin().indices().prepareRefresh("test").execute().actionGet();
for (int i = 0; i < numQueries; i++) {
String value = "value0";
PercolateRequestBuilder percolateRequestBuilder = client().preparePercolate().setIndices("test").setDocumentType("type")
.setPercolateDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", value).endObject()));
SubAggCollectionMode aggCollectionMode = randomFrom(SubAggCollectionMode.values());
percolateRequestBuilder.addAggregation(AggregationBuilders.terms("terms").field("field2").collectMode(aggCollectionMode)
.order(Order.term(true)).shardSize(2).size(1));
if (randomBoolean()) {
percolateRequestBuilder.setPercolateQuery(matchAllQuery());
}
if (randomBoolean()) {
percolateRequestBuilder.setScore(true);
} else {
percolateRequestBuilder.setSortByScore(true).setSize(numQueries);
}
boolean countOnly = randomBoolean();
if (countOnly) {
percolateRequestBuilder.setOnlyCount(countOnly);
}
percolateRequestBuilder.addAggregation(ReducerBuilders.maxBucket("max_terms").setBucketsPaths("terms>_count"));
PercolateResponse response = percolateRequestBuilder.execute().actionGet();
assertMatchCount(response, numQueries);
if (!countOnly) {
assertThat(response.getMatches(), arrayWithSize(numQueries));
}
Aggregations aggregations = response.getAggregations();
assertThat(aggregations.asList().size(), equalTo(2));
Terms terms = aggregations.get("terms");
assertThat(terms, notNullValue());
assertThat(terms.getName(), equalTo("terms"));
List<Terms.Bucket> buckets = new ArrayList<>(terms.getBuckets());
assertThat(buckets.size(), equalTo(1));
assertThat(buckets.get(0).getKeyAsString(), equalTo("a"));
InternalBucketMetricValue maxA = aggregations.get("max_terms");
assertThat(maxA, notNullValue());
assertThat(maxA.getName(), equalTo("max_terms"));
assertThat(maxA.keys(), equalTo(new String[] { "a" }));
}
}
} }