allow to execute term facet on more than one field
This commit is contained in:
parent
31d94b19a1
commit
22ed71e2e6
|
@ -239,6 +239,15 @@ public final class XContentBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public XContentBuilder field(String name, String... value) throws IOException {
|
||||
startArray(name);
|
||||
for (String o : value) {
|
||||
value(o);
|
||||
}
|
||||
endArray();
|
||||
return this;
|
||||
}
|
||||
|
||||
public XContentBuilder field(String name, Object... value) throws IOException {
|
||||
startArray(name);
|
||||
for (Object o : value) {
|
||||
|
|
|
@ -34,6 +34,7 @@ import java.util.Map;
|
|||
*/
|
||||
public class TermsFacetBuilder extends AbstractFacetBuilder {
|
||||
private String fieldName;
|
||||
private String[] fieldsNames;
|
||||
private int size = 10;
|
||||
private String[] exclude;
|
||||
private String regex;
|
||||
|
@ -62,6 +63,11 @@ public class TermsFacetBuilder extends AbstractFacetBuilder {
|
|||
return this;
|
||||
}
|
||||
|
||||
public TermsFacetBuilder fields(String... fields) {
|
||||
this.fieldsNames = fields;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TermsFacetBuilder exclude(String... exclude) {
|
||||
this.exclude = exclude;
|
||||
return this;
|
||||
|
@ -109,13 +115,21 @@ public class TermsFacetBuilder extends AbstractFacetBuilder {
|
|||
}
|
||||
|
||||
@Override public void toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
if (fieldName == null) {
|
||||
if (fieldName == null && fieldsNames == null) {
|
||||
throw new SearchSourceBuilderException("field must be set on terms facet for facet [" + name + "]");
|
||||
}
|
||||
builder.startObject(name);
|
||||
|
||||
builder.startObject(TermsFacetCollectorParser.NAME);
|
||||
builder.field("field", fieldName);
|
||||
if (fieldsNames != null) {
|
||||
if (fieldsNames.length == 1) {
|
||||
builder.field("field", fieldsNames[0]);
|
||||
} else {
|
||||
builder.field("fields", fieldsNames);
|
||||
}
|
||||
} else {
|
||||
builder.field("field", fieldName);
|
||||
}
|
||||
builder.field("size", size);
|
||||
if (exclude != null) {
|
||||
builder.startArray("exclude");
|
||||
|
|
|
@ -47,7 +47,7 @@ import java.util.regex.Pattern;
|
|||
*/
|
||||
public class TermsFacetCollector extends AbstractFacetCollector {
|
||||
|
||||
private static ThreadLocal<ThreadLocals.CleanableValue<Deque<TObjectIntHashMap<String>>>> cache = new ThreadLocal<ThreadLocals.CleanableValue<Deque<TObjectIntHashMap<String>>>>() {
|
||||
static ThreadLocal<ThreadLocals.CleanableValue<Deque<TObjectIntHashMap<String>>>> cache = new ThreadLocal<ThreadLocals.CleanableValue<Deque<TObjectIntHashMap<String>>>>() {
|
||||
@Override protected ThreadLocals.CleanableValue<Deque<TObjectIntHashMap<String>>> initialValue() {
|
||||
return new ThreadLocals.CleanableValue<Deque<TObjectIntHashMap<java.lang.String>>>(new ArrayDeque<TObjectIntHashMap<String>>());
|
||||
}
|
||||
|
@ -139,7 +139,7 @@ public class TermsFacetCollector extends AbstractFacetCollector {
|
|||
}
|
||||
}
|
||||
|
||||
private TObjectIntHashMap<String> popFacets() {
|
||||
static TObjectIntHashMap<String> popFacets() {
|
||||
Deque<TObjectIntHashMap<String>> deque = cache.get().get();
|
||||
if (deque.isEmpty()) {
|
||||
deque.add(new TObjectIntHashMap<String>());
|
||||
|
@ -149,7 +149,7 @@ public class TermsFacetCollector extends AbstractFacetCollector {
|
|||
return facets;
|
||||
}
|
||||
|
||||
private void pushFacets(TObjectIntHashMap<String> facets) {
|
||||
static void pushFacets(TObjectIntHashMap<String> facets) {
|
||||
facets.clear();
|
||||
Deque<TObjectIntHashMap<String>> deque = cache.get().get();
|
||||
if (deque != null) {
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.search.facets.terms;
|
||||
|
||||
import org.elasticsearch.common.collect.ImmutableSet;
|
||||
import org.elasticsearch.common.collect.Lists;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.search.facets.collector.FacetCollector;
|
||||
|
@ -27,6 +28,7 @@ import org.elasticsearch.search.facets.collector.FacetCollectorParser;
|
|||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
@ -46,6 +48,7 @@ public class TermsFacetCollectorParser implements FacetCollectorParser {
|
|||
int size = 10;
|
||||
|
||||
String fieldName = null;
|
||||
String[] fieldsNames = null;
|
||||
XContentParser.Token token;
|
||||
ImmutableSet<String> excluded = ImmutableSet.of();
|
||||
String regex = null;
|
||||
|
@ -68,6 +71,12 @@ public class TermsFacetCollectorParser implements FacetCollectorParser {
|
|||
builder.add(parser.text());
|
||||
}
|
||||
excluded = builder.build();
|
||||
} else if ("fields".equals(fieldName)) {
|
||||
List<String> fields = Lists.newArrayListWithCapacity(4);
|
||||
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
|
||||
fields.add(parser.text());
|
||||
}
|
||||
fieldsNames = fields.toArray(new String[fields.size()]);
|
||||
}
|
||||
} else if (token.isValue()) {
|
||||
if ("field".equals(fieldName)) {
|
||||
|
@ -96,6 +105,9 @@ public class TermsFacetCollectorParser implements FacetCollectorParser {
|
|||
if (regex != null) {
|
||||
pattern = Regex.compile(regex, regexFlags);
|
||||
}
|
||||
if (fieldsNames != null) {
|
||||
return new TermsFieldsFacetCollector(facetName, fieldsNames, size, comparatorType, context, excluded, pattern, scriptLang, script, params);
|
||||
}
|
||||
return new TermsFacetCollector(facetName, field, size, comparatorType, context, excluded, pattern, scriptLang, script, params);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,203 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search 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.facets.terms;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.elasticsearch.common.collect.BoundedTreeSet;
|
||||
import org.elasticsearch.common.collect.ImmutableList;
|
||||
import org.elasticsearch.common.collect.ImmutableSet;
|
||||
import org.elasticsearch.common.collect.Maps;
|
||||
import org.elasticsearch.common.trove.TObjectIntHashMap;
|
||||
import org.elasticsearch.common.trove.TObjectIntIterator;
|
||||
import org.elasticsearch.index.cache.field.data.FieldDataCache;
|
||||
import org.elasticsearch.index.field.data.FieldData;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.script.search.SearchScript;
|
||||
import org.elasticsearch.search.facets.Facet;
|
||||
import org.elasticsearch.search.facets.support.AbstractFacetCollector;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.elasticsearch.common.Strings.*;
|
||||
|
||||
/**
|
||||
* @author kimchy (shay.banon)
|
||||
*/
|
||||
public class TermsFieldsFacetCollector extends AbstractFacetCollector {
|
||||
|
||||
private final FieldDataCache fieldDataCache;
|
||||
|
||||
private final String[] fieldsNames;
|
||||
|
||||
private final String[] indexFieldsNames;
|
||||
|
||||
private final InternalTermsFacet.ComparatorType comparatorType;
|
||||
|
||||
private final int size;
|
||||
|
||||
private final int numberOfShards;
|
||||
|
||||
private final FieldData.Type[] fieldsDataType;
|
||||
|
||||
private FieldData[] fieldsData;
|
||||
|
||||
private final StaticAggregatorValueProc aggregator;
|
||||
|
||||
private final SearchScript script;
|
||||
|
||||
public TermsFieldsFacetCollector(String facetName, String[] fieldsNames, int size, InternalTermsFacet.ComparatorType comparatorType, SearchContext context,
|
||||
ImmutableSet<String> excluded, Pattern pattern, String scriptLang, String script, Map<String, Object> params) {
|
||||
super(facetName);
|
||||
this.fieldDataCache = context.fieldDataCache();
|
||||
this.size = size;
|
||||
this.comparatorType = comparatorType;
|
||||
this.numberOfShards = context.numberOfShards();
|
||||
|
||||
this.fieldsNames = fieldsNames;
|
||||
|
||||
fieldsDataType = new FieldData.Type[fieldsNames.length];
|
||||
fieldsData = new FieldData[fieldsNames.length];
|
||||
indexFieldsNames = new String[fieldsNames.length];
|
||||
|
||||
for (int i = 0; i < fieldsNames.length; i++) {
|
||||
MapperService.SmartNameFieldMappers smartMappers = context.mapperService().smartName(fieldsNames[i]);
|
||||
if (smartMappers == null || !smartMappers.hasMapper()) {
|
||||
this.indexFieldsNames[i] = fieldsNames[i];
|
||||
this.fieldsDataType[i] = FieldData.Type.STRING;
|
||||
} else {
|
||||
this.indexFieldsNames[i] = smartMappers.mapper().names().indexName();
|
||||
this.fieldsDataType[i] = smartMappers.mapper().fieldDataType();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (script != null) {
|
||||
this.script = new SearchScript(context.scriptSearchLookup(), scriptLang, script, params, context.scriptService());
|
||||
} else {
|
||||
this.script = null;
|
||||
}
|
||||
|
||||
if (excluded.isEmpty() && pattern == null && this.script == null) {
|
||||
aggregator = new StaticAggregatorValueProc(TermsFacetCollector.popFacets());
|
||||
} else {
|
||||
aggregator = new AggregatorValueProc(TermsFacetCollector.popFacets(), excluded, pattern, this.script);
|
||||
}
|
||||
}
|
||||
|
||||
@Override protected void doSetNextReader(IndexReader reader, int docBase) throws IOException {
|
||||
for (int i = 0; i < fieldsNames.length; i++) {
|
||||
fieldsData[i] = fieldDataCache.cache(fieldsDataType[i], reader, indexFieldsNames[i]);
|
||||
}
|
||||
if (script != null) {
|
||||
script.setNextReader(reader);
|
||||
}
|
||||
}
|
||||
|
||||
@Override protected void doCollect(int doc) throws IOException {
|
||||
for (FieldData fieldData : fieldsData) {
|
||||
fieldData.forEachValueInDoc(doc, aggregator);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public Facet facet() {
|
||||
TObjectIntHashMap<String> facets = aggregator.facets();
|
||||
if (facets.isEmpty()) {
|
||||
TermsFacetCollector.pushFacets(facets);
|
||||
return new InternalTermsFacet(facetName, arrayToCommaDelimitedString(fieldsNames), comparatorType, size, ImmutableList.<InternalTermsFacet.Entry>of());
|
||||
} else {
|
||||
// we need to fetch facets of "size * numberOfShards" because of problems in how they are distributed across shards
|
||||
BoundedTreeSet<InternalTermsFacet.Entry> ordered = new BoundedTreeSet<InternalTermsFacet.Entry>(InternalTermsFacet.ComparatorType.COUNT.comparator(), size * numberOfShards);
|
||||
for (TObjectIntIterator<String> it = facets.iterator(); it.hasNext();) {
|
||||
it.advance();
|
||||
ordered.add(new InternalTermsFacet.Entry(it.key(), it.value()));
|
||||
}
|
||||
TermsFacetCollector.pushFacets(facets);
|
||||
return new InternalTermsFacet(facetName, arrayToCommaDelimitedString(fieldsNames), comparatorType, size, ordered);
|
||||
}
|
||||
}
|
||||
|
||||
public static class AggregatorValueProc extends StaticAggregatorValueProc {
|
||||
|
||||
private final ImmutableSet<String> excluded;
|
||||
|
||||
private final Matcher matcher;
|
||||
|
||||
private final SearchScript script;
|
||||
|
||||
private final Map<String, Object> scriptParams;
|
||||
|
||||
public AggregatorValueProc(TObjectIntHashMap<String> facets, ImmutableSet<String> excluded, Pattern pattern, SearchScript script) {
|
||||
super(facets);
|
||||
this.excluded = excluded;
|
||||
this.matcher = pattern != null ? pattern.matcher("") : null;
|
||||
this.script = script;
|
||||
if (script != null) {
|
||||
scriptParams = Maps.newHashMapWithExpectedSize(4);
|
||||
} else {
|
||||
scriptParams = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override public void onValue(int docId, String value) {
|
||||
if (excluded != null && excluded.contains(value)) {
|
||||
return;
|
||||
}
|
||||
if (matcher != null && !matcher.reset(value).matches()) {
|
||||
return;
|
||||
}
|
||||
if (script != null) {
|
||||
scriptParams.put("term", value);
|
||||
Object scriptValue = script.execute(docId, scriptParams);
|
||||
if (scriptValue == null) {
|
||||
return;
|
||||
}
|
||||
if (scriptValue instanceof Boolean) {
|
||||
if (!((Boolean) scriptValue)) {
|
||||
return;
|
||||
}
|
||||
} else {
|
||||
value = scriptValue.toString();
|
||||
}
|
||||
}
|
||||
super.onValue(docId, value);
|
||||
}
|
||||
}
|
||||
|
||||
public static class StaticAggregatorValueProc implements FieldData.StringValueInDocProc {
|
||||
|
||||
private final TObjectIntHashMap<String> facets;
|
||||
|
||||
public StaticAggregatorValueProc(TObjectIntHashMap<String> facets) {
|
||||
this.facets = facets;
|
||||
}
|
||||
|
||||
@Override public void onValue(int docId, String value) {
|
||||
facets.adjustOrPutValue(value, 1, 1);
|
||||
}
|
||||
|
||||
public final TObjectIntHashMap<String> facets() {
|
||||
return facets;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -368,6 +368,25 @@ public class SimpleFacetsTests extends AbstractNodesTests {
|
|||
assertThat(facet.entries().get(0).count(), equalTo(2));
|
||||
assertThat(facet.entries().get(1).term(), equalTo("zzz"));
|
||||
assertThat(facet.entries().get(1).count(), equalTo(1));
|
||||
|
||||
// Fields
|
||||
|
||||
searchResponse = client.prepareSearch()
|
||||
.setQuery(matchAllQuery())
|
||||
.addFacet(termsFacet("facet1").fields("stag", "tag").size(10))
|
||||
.execute().actionGet();
|
||||
|
||||
facet = searchResponse.facets().facet("facet1");
|
||||
assertThat(facet.name(), equalTo("facet1"));
|
||||
assertThat(facet.entries().size(), equalTo(4));
|
||||
assertThat(facet.entries().get(0).term(), anyOf(equalTo("111"), equalTo("yyy")));
|
||||
assertThat(facet.entries().get(0).count(), equalTo(2));
|
||||
assertThat(facet.entries().get(1).term(), anyOf(equalTo("111"), equalTo("yyy")));
|
||||
assertThat(facet.entries().get(1).count(), equalTo(2));
|
||||
assertThat(facet.entries().get(2).term(), anyOf(equalTo("zzz"), equalTo("xxx")));
|
||||
assertThat(facet.entries().get(2).count(), equalTo(1));
|
||||
assertThat(facet.entries().get(3).term(), anyOf(equalTo("zzz"), equalTo("xxx")));
|
||||
assertThat(facet.entries().get(3).count(), equalTo(1));
|
||||
}
|
||||
|
||||
@Test public void testTermFacetWithEqualTermDistribution() throws Exception {
|
||||
|
|
Loading…
Reference in New Issue