Do not assume term queries use the inverted index. #17532

We have a couple places in the code base that assume that search is always done
on the inverted index. However with the new points API in Lucene 6, this is not
true anymore. This commit makes MappedFieldType.indexedValueForSearch protected
and fixes call sites to keep working for field types that use the inverted
index and either work differently ar throw an exception otherwise. For instance,
it will still be possible to run cross_fields multi match queries on numeric
fields, but the score contributions will not be blended as well as before, and
significant terms aggregations on long terms will not be possible anymore since
points do not record document frequencies.
This commit is contained in:
Adrien Grand 2016-04-05 11:04:48 +02:00
parent 013acf9179
commit 226644ea2c
13 changed files with 216 additions and 96 deletions

View File

@ -309,23 +309,19 @@ public abstract class MappedFieldType extends FieldType {
return value; return value;
} }
/** Returns the indexed value used to construct search "values". */ /** Returns the indexed value used to construct search "values".
public BytesRef indexedValueForSearch(Object value) { * This method is used for the default implementations of most
* query factory methods such as {@link #termQuery}. */
protected BytesRef indexedValueForSearch(Object value) {
return BytesRefs.toBytesRef(value); return BytesRefs.toBytesRef(value);
} }
/** /** Generates a query that will only match documents that contain the given value.
* Creates a term associated with the field of this mapper for the given * The default implementation returns a {@link TermQuery} over the value bytes,
* value. Its important to use termQuery when building term queries because * boosted by {@link #boost()}.
* things like ParentFieldMapper override it to make more interesting * @throws IllegalArgumentException if {@code value} cannot be converted to the expected data type */
* queries.
*/
protected Term createTerm(Object value) {
return new Term(name(), indexedValueForSearch(value));
}
public Query termQuery(Object value, @Nullable QueryShardContext context) { public Query termQuery(Object value, @Nullable QueryShardContext context) {
TermQuery query = new TermQuery(createTerm(value)); TermQuery query = new TermQuery(new Term(name(), indexedValueForSearch(value)));
if (boost == 1f || if (boost == 1f ||
(context != null && context.indexVersionCreated().before(Version.V_5_0_0_alpha1))) { (context != null && context.indexVersionCreated().before(Version.V_5_0_0_alpha1))) {
return query; return query;
@ -349,11 +345,12 @@ public abstract class MappedFieldType extends FieldType {
} }
public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) { public Query fuzzyQuery(Object value, Fuzziness fuzziness, int prefixLength, int maxExpansions, boolean transpositions) {
return new FuzzyQuery(createTerm(value), fuzziness.asDistance(BytesRefs.toString(value)), prefixLength, maxExpansions, transpositions); return new FuzzyQuery(new Term(name(), indexedValueForSearch(value)),
fuzziness.asDistance(BytesRefs.toString(value)), prefixLength, maxExpansions, transpositions);
} }
public Query prefixQuery(String value, @Nullable MultiTermQuery.RewriteMethod method, @Nullable QueryShardContext context) { public Query prefixQuery(String value, @Nullable MultiTermQuery.RewriteMethod method, @Nullable QueryShardContext context) {
PrefixQuery query = new PrefixQuery(createTerm(value)); PrefixQuery query = new PrefixQuery(new Term(name(), indexedValueForSearch(value)));
if (method != null) { if (method != null) {
query.setRewriteMethod(method); query.setRewriteMethod(method);
} }
@ -365,7 +362,7 @@ public abstract class MappedFieldType extends FieldType {
throw new QueryShardException(context, "Cannot use regular expression to filter numeric field [" + name + "]"); throw new QueryShardException(context, "Cannot use regular expression to filter numeric field [" + name + "]");
} }
RegexpQuery query = new RegexpQuery(createTerm(value), flags, maxDeterminizedStates); RegexpQuery query = new RegexpQuery(new Term(name(), indexedValueForSearch(value)), flags, maxDeterminizedStates);
if (method != null) { if (method != null) {
query.setRewriteMethod(method); query.setRewriteMethod(method);
} }
@ -453,4 +450,19 @@ public abstract class MappedFieldType extends FieldType {
return DocValueFormat.RAW; return DocValueFormat.RAW;
} }
/**
* Extract a {@link Term} from a query created with {@link #termQuery} by
* recursively removing {@link BoostQuery} wrappers.
* @throws IllegalArgumentException if the wrapped query is not a {@link TermQuery}
*/
public static Term extractTerm(Query termQuery) {
while (termQuery instanceof BoostQuery) {
termQuery = ((BoostQuery) termQuery).getQuery();
}
if (termQuery instanceof TermQuery == false) {
throw new IllegalArgumentException("Cannot extract a term from a query of type "
+ termQuery.getClass() + ": " + termQuery);
}
return ((TermQuery) termQuery).getTerm();
}
} }

View File

@ -20,6 +20,7 @@ package org.elasticsearch.index.mapper.core;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.suggest.document.Completion50PostingsFormat; import org.apache.lucene.search.suggest.document.Completion50PostingsFormat;
import org.apache.lucene.search.suggest.document.CompletionAnalyzer; import org.apache.lucene.search.suggest.document.CompletionAnalyzer;
import org.apache.lucene.search.suggest.document.CompletionQuery; import org.apache.lucene.search.suggest.document.CompletionQuery;
@ -265,14 +266,14 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
* Completion prefix query * Completion prefix query
*/ */
public CompletionQuery prefixQuery(Object value) { public CompletionQuery prefixQuery(Object value) {
return new PrefixCompletionQuery(searchAnalyzer().analyzer(), createTerm(value)); return new PrefixCompletionQuery(searchAnalyzer().analyzer(), new Term(name(), indexedValueForSearch(value)));
} }
/** /**
* Completion prefix regular expression query * Completion prefix regular expression query
*/ */
public CompletionQuery regexpQuery(Object value, int flags, int maxDeterminizedStates) { public CompletionQuery regexpQuery(Object value, int flags, int maxDeterminizedStates) {
return new RegexCompletionQuery(createTerm(value), flags, maxDeterminizedStates); return new RegexCompletionQuery(new Term(name(), indexedValueForSearch(value)), flags, maxDeterminizedStates);
} }
/** /**
@ -281,7 +282,7 @@ public class CompletionFieldMapper extends FieldMapper implements ArrayValueMapp
public CompletionQuery fuzzyQuery(String value, Fuzziness fuzziness, int nonFuzzyPrefixLength, public CompletionQuery fuzzyQuery(String value, Fuzziness fuzziness, int nonFuzzyPrefixLength,
int minFuzzyPrefixLength, int maxExpansions, boolean transpositions, int minFuzzyPrefixLength, int maxExpansions, boolean transpositions,
boolean unicodeAware) { boolean unicodeAware) {
return new FuzzyCompletionQuery(searchAnalyzer().analyzer(), createTerm(value), null, return new FuzzyCompletionQuery(searchAnalyzer().analyzer(), new Term(name(), indexedValueForSearch(value)), null,
fuzziness.asDistance(), transpositions, nonFuzzyPrefixLength, minFuzzyPrefixLength, fuzziness.asDistance(), transpositions, nonFuzzyPrefixLength, minFuzzyPrefixLength,
unicodeAware, maxExpansions); unicodeAware, maxExpansions);
} }

View File

@ -204,7 +204,7 @@ public class AllFieldMapper extends MetadataFieldMapper {
@Override @Override
public Query termQuery(Object value, QueryShardContext context) { public Query termQuery(Object value, QueryShardContext context) {
return queryStringTermQuery(createTerm(value)); return queryStringTermQuery(new Term(name(), indexedValueForSearch(value)));
} }
} }

View File

@ -185,10 +185,6 @@ public class UidFieldMapper extends MetadataFieldMapper {
} }
} }
public Term term(String uid) {
return new Term(fieldType().name(), fieldType().indexedValueForSearch(uid));
}
@Override @Override
protected String contentType() { protected String contentType() {
return CONTENT_TYPE; return CONTENT_TYPE;

View File

@ -20,9 +20,9 @@
package org.elasticsearch.index.query; package org.elasticsearch.index.query;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.spans.SpanQuery; import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery; import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.lucene.BytesRefs;
@ -75,17 +75,15 @@ public class SpanTermQueryBuilder extends BaseTermQueryBuilder<SpanTermQueryBuil
@Override @Override
protected SpanQuery doToQuery(QueryShardContext context) throws IOException { protected SpanQuery doToQuery(QueryShardContext context) throws IOException {
BytesRef valueBytes = null;
String fieldName = this.fieldName;
MappedFieldType mapper = context.fieldMapper(fieldName); MappedFieldType mapper = context.fieldMapper(fieldName);
if (mapper != null) { Term term;
fieldName = mapper.name(); if (mapper == null) {
valueBytes = mapper.indexedValueForSearch(value); term = new Term(fieldName, BytesRefs.toBytesRef(value));
} else {
Query termQuery = mapper.termQuery(value, context);
term = MappedFieldType.extractTerm(termQuery);
} }
if (valueBytes == null) { return new SpanTermQuery(term);
valueBytes = BytesRefs.toBytesRef(this.value);
}
return new SpanTermQuery(new Term(fieldName, valueBytes));
} }
public static SpanTermQueryBuilder fromXContent(QueryParseContext parseContext) throws IOException, ParsingException { public static SpanTermQueryBuilder fromXContent(QueryParseContext parseContext) throws IOException, ParsingException {

View File

@ -23,12 +23,12 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.WildcardQuery; import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
@ -172,19 +172,16 @@ public class WildcardQueryBuilder extends AbstractQueryBuilder<WildcardQueryBuil
@Override @Override
protected Query doToQuery(QueryShardContext context) throws IOException { protected Query doToQuery(QueryShardContext context) throws IOException {
String indexFieldName;
BytesRef valueBytes;
MappedFieldType fieldType = context.fieldMapper(fieldName); MappedFieldType fieldType = context.fieldMapper(fieldName);
if (fieldType != null) { Term term;
indexFieldName = fieldType.name(); if (fieldType == null) {
valueBytes = fieldType.indexedValueForSearch(value); term = new Term(fieldName, BytesRefs.toBytesRef(value));
} else { } else {
indexFieldName = fieldName; Query termQuery = fieldType.termQuery(value, context);
valueBytes = new BytesRef(value); term = MappedFieldType.extractTerm(termQuery);
} }
WildcardQuery query = new WildcardQuery(new Term(indexFieldName, valueBytes)); WildcardQuery query = new WildcardQuery(term);
MultiTermQuery.RewriteMethod rewriteMethod = QueryParsers.parseRewriteMethod(context.parseFieldMatcher(), rewrite, null); MultiTermQuery.RewriteMethod rewriteMethod = QueryParsers.parseRewriteMethod(context.parseFieldMatcher(), rewrite, null);
QueryParsers.setRewriteMethod(query, rewriteMethod); QueryParsers.setRewriteMethod(query, rewriteMethod);
return query; return query;

View File

@ -27,6 +27,9 @@ import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.DisjunctionMaxQuery; import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.search.BooleanClause.Occur;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
@ -36,9 +39,11 @@ import org.elasticsearch.index.query.QueryShardContext;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
public class MultiMatchQuery extends MatchQuery { public class MultiMatchQuery extends MatchQuery {
@ -153,7 +158,7 @@ public class MultiMatchQuery extends MatchQuery {
} }
} }
public class CrossFieldsQueryBuilder extends QueryBuilder { final class CrossFieldsQueryBuilder extends QueryBuilder {
private FieldAndFieldType[] blendedFields; private FieldAndFieldType[] blendedFields;
public CrossFieldsQueryBuilder(float tieBreaker) { public CrossFieldsQueryBuilder(float tieBreaker) {
@ -175,7 +180,7 @@ public class MultiMatchQuery extends MatchQuery {
} }
Float boost = entry.getValue(); Float boost = entry.getValue();
boost = boost == null ? Float.valueOf(1.0f) : boost; boost = boost == null ? Float.valueOf(1.0f) : boost;
groups.get(actualAnalyzer).add(new FieldAndFieldType(name, fieldType, boost)); groups.get(actualAnalyzer).add(new FieldAndFieldType(fieldType, boost));
} else { } else {
missing.add(new Tuple<>(name, entry.getValue())); missing.add(new Tuple<>(name, entry.getValue()));
} }
@ -203,7 +208,7 @@ public class MultiMatchQuery extends MatchQuery {
* we just pick the first field. It shouldn't matter because * we just pick the first field. It shouldn't matter because
* fields are already grouped by their analyzers/types. * fields are already grouped by their analyzers/types.
*/ */
String representativeField = group.get(0).field; String representativeField = group.get(0).fieldType.name();
Query q = parseGroup(type.matchQueryType(), representativeField, 1f, value, minimumShouldMatch); Query q = parseGroup(type.matchQueryType(), representativeField, 1f, value, minimumShouldMatch);
if (q != null) { if (q != null) {
queries.add(q); queries.add(q);
@ -218,20 +223,7 @@ public class MultiMatchQuery extends MatchQuery {
if (blendedFields == null) { if (blendedFields == null) {
return super.blendTerm(term, fieldType); return super.blendTerm(term, fieldType);
} }
final Term[] terms = new Term[blendedFields.length]; return MultiMatchQuery.blendTerm(term.bytes(), commonTermsCutoff, tieBreaker, blendedFields);
float[] blendedBoost = new float[blendedFields.length];
for (int i = 0; i < blendedFields.length; i++) {
terms[i] = blendedFields[i].newTerm(term.text());
blendedBoost[i] = blendedFields[i].boost;
}
if (commonTermsCutoff != null) {
return BlendedTermQuery.commonTermsBlendedQuery(terms, blendedBoost, false, commonTermsCutoff);
}
if (tieBreaker == 1.0f) {
return BlendedTermQuery.booleanBlendedQuery(terms, blendedBoost, false);
}
return BlendedTermQuery.dismaxBlendedQuery(terms, blendedBoost, tieBreaker);
} }
@Override @Override
@ -245,6 +237,64 @@ public class MultiMatchQuery extends MatchQuery {
} }
} }
static Query blendTerm(BytesRef value, Float commonTermsCutoff, float tieBreaker, FieldAndFieldType... blendedFields) {
List<Query> queries = new ArrayList<>();
Term[] terms = new Term[blendedFields.length];
float[] blendedBoost = new float[blendedFields.length];
int i = 0;
for (FieldAndFieldType ft : blendedFields) {
Query query;
try {
query = ft.fieldType.termQuery(value, null);
} catch (IllegalArgumentException e) {
// the query expects a certain class of values such as numbers
// of ip addresses and the value can't be parsed, so ignore this
// field
continue;
}
float boost = ft.boost;
while (query instanceof BoostQuery) {
BoostQuery bq = (BoostQuery) query;
query = bq.getQuery();
boost *= bq.getBoost();
}
if (query.getClass() == TermQuery.class) {
terms[i] = ((TermQuery) query).getTerm();
blendedBoost[i] = boost;
i++;
} else {
if (boost != 1f) {
query = new BoostQuery(query, boost);
}
queries.add(query);
}
}
if (i > 0) {
terms = Arrays.copyOf(terms, i);
blendedBoost = Arrays.copyOf(blendedBoost, i);
if (commonTermsCutoff != null) {
queries.add(BlendedTermQuery.commonTermsBlendedQuery(terms, blendedBoost, false, commonTermsCutoff));
} else if (tieBreaker == 1.0f) {
queries.add(BlendedTermQuery.booleanBlendedQuery(terms, blendedBoost, false));
} else {
queries.add(BlendedTermQuery.dismaxBlendedQuery(terms, blendedBoost, tieBreaker));
}
}
if (queries.size() == 1) {
return queries.get(0);
} else {
// best effort: add clauses that are not term queries so that they have an opportunity to match
// however their score contribution will be different
// TODO: can we improve this?
BooleanQuery.Builder bq = new BooleanQuery.Builder();
bq.setDisableCoord(true);
for (Query query : queries) {
bq.add(query, Occur.SHOULD);
}
return bq.build();
}
}
@Override @Override
protected Query blendTermQuery(Term term, MappedFieldType fieldType) { protected Query blendTermQuery(Term term, MappedFieldType fieldType) {
if (queryBuilder == null) { if (queryBuilder == null) {
@ -262,31 +312,13 @@ public class MultiMatchQuery extends MatchQuery {
return queryBuilder.termQuery(fieldType, value); return queryBuilder.termQuery(fieldType, value);
} }
private static final class FieldAndFieldType { static final class FieldAndFieldType {
final String field;
final MappedFieldType fieldType; final MappedFieldType fieldType;
final float boost; final float boost;
FieldAndFieldType(MappedFieldType fieldType, float boost) {
private FieldAndFieldType(String field, MappedFieldType fieldType, float boost) { this.fieldType = Objects.requireNonNull(fieldType);
this.field = field;
this.fieldType = fieldType;
this.boost = boost; this.boost = boost;
} }
public Term newTerm(String value) {
try {
/*
* Note that this ignore any overrides the fieldType might do
* for termQuery, meaning things like _parent won't work here.
*/
return new Term(fieldType.name(), fieldType.indexedValueForSearch(value));
} catch (RuntimeException ex) {
// we can't parse it just use the incoming value -- it will
// just have a DF of 0 at the end of the day and will be ignored
// Note that this is like lenient = true always
}
return new Term(field, value);
}
} }
} }

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
import org.apache.lucene.index.SnapshotDeletionPolicy; import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.UsageTrackingQueryCachingPolicy; import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
@ -77,6 +78,7 @@ import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.get.ShardGetService;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperForType; import org.elasticsearch.index.mapper.DocumentMapperForType;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.SourceToParse;
@ -473,7 +475,10 @@ public class IndexShard extends AbstractIndexShardComponent {
if (docMapper.getMapping() != null) { if (docMapper.getMapping() != null) {
doc.addDynamicMappingsUpdate(docMapper.getMapping()); doc.addDynamicMappingsUpdate(docMapper.getMapping());
} }
return new Engine.Index(docMapper.getDocumentMapper().uidMapper().term(doc.uid().stringValue()), doc, version, versionType, origin, startTime); MappedFieldType uidFieldType = docMapper.getDocumentMapper().uidMapper().fieldType();
Query uidQuery = uidFieldType.termQuery(doc.uid().stringValue(), null);
Term uid = MappedFieldType.extractTerm(uidQuery);
return new Engine.Index(uid, doc, version, versionType, origin, startTime);
} }
/** /**
@ -509,12 +514,18 @@ public class IndexShard extends AbstractIndexShardComponent {
public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version, VersionType versionType) { public Engine.Delete prepareDeleteOnPrimary(String type, String id, long version, VersionType versionType) {
verifyPrimary(); verifyPrimary();
final DocumentMapper documentMapper = docMapper(type).getDocumentMapper(); final DocumentMapper documentMapper = docMapper(type).getDocumentMapper();
return prepareDelete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, Engine.Operation.Origin.PRIMARY); final MappedFieldType uidFieldType = documentMapper.uidMapper().fieldType();
final Query uidQuery = uidFieldType.termQuery(Uid.createUid(type, id), null);
final Term uid = MappedFieldType.extractTerm(uidQuery);
return prepareDelete(type, id, uid, version, versionType, Engine.Operation.Origin.PRIMARY);
} }
public Engine.Delete prepareDeleteOnReplica(String type, String id, long version, VersionType versionType) { public Engine.Delete prepareDeleteOnReplica(String type, String id, long version, VersionType versionType) {
final DocumentMapper documentMapper = docMapper(type).getDocumentMapper(); final DocumentMapper documentMapper = docMapper(type).getDocumentMapper();
return prepareDelete(type, id, documentMapper.uidMapper().term(Uid.createUid(type, id)), version, versionType, Engine.Operation.Origin.REPLICA); final MappedFieldType uidFieldType = documentMapper.uidMapper().fieldType();
final Query uidQuery = uidFieldType.termQuery(Uid.createUid(type, id), null);
final Term uid = MappedFieldType.extractTerm(uidQuery);
return prepareDelete(type, id, uid, version, versionType, Engine.Operation.Origin.REPLICA);
} }
static Engine.Delete prepareDelete(String type, String id, Term uid, long version, VersionType versionType, Engine.Operation.Origin origin) { static Engine.Delete prepareDelete(String type, String id, Term uid, long version, VersionType versionType, Engine.Operation.Origin origin) {

View File

@ -21,6 +21,7 @@ package org.elasticsearch.search.aggregations.bucket.significant;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.PostingsEnum; import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
@ -145,8 +146,10 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
return result; return result;
} }
public long getBackgroundFrequency(long term) { public long getBackgroundFrequency(long value) {
BytesRef indexedVal = fieldType.indexedValueForSearch(term); Query query = fieldType.termQuery(value, null);
Term term = MappedFieldType.extractTerm(query);
BytesRef indexedVal = term.bytes();
return getBackgroundFrequency(indexedVal); return getBackgroundFrequency(indexedVal);
} }
@ -256,7 +259,6 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic, AggregationContext aggregationContext, Aggregator parent, SignificanceHeuristic significanceHeuristic,
SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators, SignificantTermsAggregatorFactory termsAggregatorFactory, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) throws IOException { Map<String, Object> metaData) throws IOException {
ValuesSource.Bytes.WithOrdinals valueSourceWithOrdinals = (ValuesSource.Bytes.WithOrdinals) valuesSource;
final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter(); final IncludeExclude.OrdinalsFilter filter = includeExclude == null ? null : includeExclude.convertToOrdinalsFilter();
return new GlobalOrdinalsSignificantTermsAggregator(name, factories, return new GlobalOrdinalsSignificantTermsAggregator(name, factories,
(ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, format, bucketCountThresholds, filter, (ValuesSource.Bytes.WithOrdinals.FieldData) valuesSource, format, bucketCountThresholds, filter,

View File

@ -19,9 +19,10 @@
package org.elasticsearch.index.query; package org.elasticsearch.index.query;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.spans.SpanTermQuery; import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
@ -44,8 +45,8 @@ public class SpanTermQueryBuilderTests extends AbstractTermQueryTestCase<SpanTer
assertThat(spanTermQuery.getTerm().field(), equalTo(queryBuilder.fieldName())); assertThat(spanTermQuery.getTerm().field(), equalTo(queryBuilder.fieldName()));
MappedFieldType mapper = context.fieldMapper(queryBuilder.fieldName()); MappedFieldType mapper = context.fieldMapper(queryBuilder.fieldName());
if (mapper != null) { if (mapper != null) {
BytesRef bytesRef = mapper.indexedValueForSearch(queryBuilder.value()); Term term = ((TermQuery) mapper.termQuery(queryBuilder.value(), null)).getTerm();
assertThat(spanTermQuery.getTerm().bytes(), equalTo(bytesRef)); assertThat(spanTermQuery.getTerm(), equalTo(term));
} else { } else {
assertThat(spanTermQuery.getTerm().bytes(), equalTo(BytesRefs.toBytesRef(queryBuilder.value()))); assertThat(spanTermQuery.getTerm().bytes(), equalTo(BytesRefs.toBytesRef(queryBuilder.value())));
} }

View File

@ -19,9 +19,9 @@
package org.elasticsearch.index.query; package org.elasticsearch.index.query;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.lucene.BytesRefs;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
@ -48,8 +48,8 @@ public class TermQueryBuilderTests extends AbstractTermQueryTestCase<TermQueryBu
assertThat(termQuery.getTerm().field(), equalTo(queryBuilder.fieldName())); assertThat(termQuery.getTerm().field(), equalTo(queryBuilder.fieldName()));
MappedFieldType mapper = context.fieldMapper(queryBuilder.fieldName()); MappedFieldType mapper = context.fieldMapper(queryBuilder.fieldName());
if (mapper != null) { if (mapper != null) {
BytesRef bytesRef = mapper.indexedValueForSearch(queryBuilder.value()); Term term = ((TermQuery) mapper.termQuery(queryBuilder.value(), null)).getTerm();
assertThat(termQuery.getTerm().bytes(), equalTo(bytesRef)); assertThat(termQuery.getTerm(), equalTo(term));
} else { } else {
assertThat(termQuery.getTerm().bytes(), equalTo(BytesRefs.toBytesRef(queryBuilder.value()))); assertThat(termQuery.getTerm().bytes(), equalTo(BytesRefs.toBytesRef(queryBuilder.value())));
} }

View File

@ -20,18 +20,24 @@
package org.elasticsearch.index.search; package org.elasticsearch.index.search;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.queries.BlendedTermQuery;
import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.DisjunctionMaxQuery; import org.apache.lucene.search.DisjunctionMaxQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MockFieldMapper.FakeFieldType;
import org.elasticsearch.index.query.MultiMatchQueryBuilder; import org.elasticsearch.index.query.MultiMatchQueryBuilder;
import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.search.MultiMatchQuery.FieldAndFieldType;
import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.ESSingleNodeTestCase;
import org.junit.Before; import org.junit.Before;
@ -83,4 +89,69 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase {
assertEquals(expected.build(), rewrittenQuery); assertEquals(expected.build(), rewrittenQuery);
} }
} }
public void testBlendTerms() {
FakeFieldType ft1 = new FakeFieldType();
ft1.setName("foo");
FakeFieldType ft2 = new FakeFieldType();
ft2.setName("bar");
Term[] terms = new Term[] { new Term("foo", "baz"), new Term("bar", "baz") };
float[] boosts = new float[] {2, 3};
Query expected = BlendedTermQuery.booleanBlendedQuery(terms, boosts, false);
Query actual = MultiMatchQuery.blendTerm(new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3));
assertEquals(expected, actual);
}
public void testBlendTermsWithFieldBoosts() {
FakeFieldType ft1 = new FakeFieldType();
ft1.setName("foo");
ft1.setBoost(100);
FakeFieldType ft2 = new FakeFieldType();
ft2.setName("bar");
ft2.setBoost(10);
Term[] terms = new Term[] { new Term("foo", "baz"), new Term("bar", "baz") };
float[] boosts = new float[] {200, 30};
Query expected = BlendedTermQuery.booleanBlendedQuery(terms, boosts, false);
Query actual = MultiMatchQuery.blendTerm(new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3));
assertEquals(expected, actual);
}
public void testBlendTermsUnsupportedValue() {
FakeFieldType ft1 = new FakeFieldType();
ft1.setName("foo");
FakeFieldType ft2 = new FakeFieldType() {
@Override
public Query termQuery(Object value, QueryShardContext context) {
throw new IllegalArgumentException();
}
};
ft2.setName("bar");
Term[] terms = new Term[] { new Term("foo", "baz") };
float[] boosts = new float[] {2};
Query expected = BlendedTermQuery.booleanBlendedQuery(terms, boosts, false);
Query actual = MultiMatchQuery.blendTerm(new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3));
assertEquals(expected, actual);
}
public void testBlendNoTermQuery() {
FakeFieldType ft1 = new FakeFieldType();
ft1.setName("foo");
FakeFieldType ft2 = new FakeFieldType() {
@Override
public Query termQuery(Object value, QueryShardContext context) {
return new MatchAllDocsQuery();
}
};
ft2.setName("bar");
Term[] terms = new Term[] { new Term("foo", "baz") };
float[] boosts = new float[] {2};
Query expectedClause1 = BlendedTermQuery.booleanBlendedQuery(terms, boosts, false);
Query expectedClause2 = new BoostQuery(new MatchAllDocsQuery(), 3);
Query expected = new BooleanQuery.Builder().setDisableCoord(true)
.add(expectedClause1, Occur.SHOULD)
.add(expectedClause2, Occur.SHOULD)
.build();
Query actual = MultiMatchQuery.blendTerm(new BytesRef("baz"), null, 1f, new FieldAndFieldType(ft1, 2), new FieldAndFieldType(ft2, 3));
assertEquals(expected, actual);
}
} }

View File

@ -24,7 +24,6 @@ import java.util.List;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
// this sucks how much must be overridden just do get a dummy field mapper... // this sucks how much must be overridden just do get a dummy field mapper...
@ -50,7 +49,7 @@ public class MockFieldMapper extends FieldMapper {
return fullName.substring(ndx + 1); return fullName.substring(ndx + 1);
} }
static class FakeFieldType extends MappedFieldType { public static class FakeFieldType extends MappedFieldType {
public FakeFieldType() { public FakeFieldType() {
} }