parent
f155525cad
commit
c449fbdd68
|
@ -29,7 +29,6 @@ import org.apache.lucene.search.*;
|
|||
import org.elasticsearch.common.io.FastStringReader;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParseContext;
|
||||
|
@ -768,9 +767,7 @@ public class MapperQueryParser extends QueryParser {
|
|||
private Collection<String> extractMultiFields(String field) {
|
||||
Collection<String> fields = null;
|
||||
if (field != null) {
|
||||
if (Regex.isSimpleMatchPattern(field)) {
|
||||
fields = parseContext.mapperService().simpleMatchToIndexNames(field);
|
||||
}
|
||||
fields = parseContext.simpleMatchToIndexNames(field);
|
||||
} else {
|
||||
fields = settings.fields();
|
||||
}
|
||||
|
|
|
@ -20,10 +20,7 @@
|
|||
package org.elasticsearch.index.mapper;
|
||||
|
||||
import com.google.common.base.Charsets;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.UnmodifiableIterator;
|
||||
import com.google.common.collect.*;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.AnalyzerWrapper;
|
||||
import org.apache.lucene.index.Term;
|
||||
|
@ -508,7 +505,40 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
|||
return fullPathObjectMappers.get(path);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the fields that match the given pattern, with an optional narrowing
|
||||
* based on a list of types.
|
||||
*/
|
||||
public Set<String> simpleMatchToIndexNames(String pattern, @Nullable String[] types) {
|
||||
if (types == null || types.length == 0) {
|
||||
return simpleMatchToIndexNames(pattern);
|
||||
}
|
||||
if (types.length == 1 && types[0].equals("_all")) {
|
||||
return simpleMatchToIndexNames(pattern);
|
||||
}
|
||||
if (!Regex.isSimpleMatchPattern(pattern)) {
|
||||
return ImmutableSet.of(pattern);
|
||||
}
|
||||
Set<String> fields = Sets.newHashSet();
|
||||
for (String type : types) {
|
||||
DocumentMapper possibleDocMapper = mappers.get(type);
|
||||
if (possibleDocMapper != null) {
|
||||
for (String indexName : possibleDocMapper.mappers().simpleMatchToIndexNames(pattern)) {
|
||||
fields.add(indexName);
|
||||
}
|
||||
}
|
||||
}
|
||||
return fields;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the fields that match the given pattern. If the pattern is prefixed with a type
|
||||
* then the fields will be returned with a type prefix.
|
||||
*/
|
||||
public Set<String> simpleMatchToIndexNames(String pattern) {
|
||||
if (!Regex.isSimpleMatchPattern(pattern)) {
|
||||
return ImmutableSet.of(pattern);
|
||||
}
|
||||
int dotIndex = pattern.indexOf('.');
|
||||
if (dotIndex != -1) {
|
||||
String possibleType = pattern.substring(0, dotIndex);
|
||||
|
@ -762,11 +792,11 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
|||
public Analyzer searchQuoteAnalyzer() {
|
||||
return this.searchQuoteAnalyzer;
|
||||
}
|
||||
|
||||
|
||||
public Analyzer fieldSearchAnalyzer(String field) {
|
||||
return this.searchAnalyzer.getWrappedAnalyzer(field);
|
||||
}
|
||||
|
||||
|
||||
public Analyzer fieldSearchQuoteAnalyzer(String field) {
|
||||
return this.searchQuoteAnalyzer.getWrappedAnalyzer(field);
|
||||
}
|
||||
|
|
|
@ -19,13 +19,17 @@
|
|||
|
||||
package org.elasticsearch.index.query;
|
||||
|
||||
import org.apache.lucene.search.BooleanClause;
|
||||
import org.apache.lucene.search.Filter;
|
||||
import org.apache.lucene.search.TermRangeFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.XBooleanFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.elasticsearch.index.query.support.QueryParsers.wrapSmartNameFilter;
|
||||
|
||||
|
@ -49,7 +53,7 @@ public class ExistsFilterParser implements FilterParser {
|
|||
public Filter parse(QueryParseContext parseContext) throws IOException, QueryParsingException {
|
||||
XContentParser parser = parseContext.parser();
|
||||
|
||||
String fieldName = null;
|
||||
String fieldPattern = null;
|
||||
String filterName = null;
|
||||
|
||||
XContentParser.Token token;
|
||||
|
@ -59,7 +63,7 @@ public class ExistsFilterParser implements FilterParser {
|
|||
currentFieldName = parser.currentName();
|
||||
} else if (token.isValue()) {
|
||||
if ("field".equals(currentFieldName)) {
|
||||
fieldName = parser.text();
|
||||
fieldPattern = parser.text();
|
||||
} else if ("_name".equals(currentFieldName)) {
|
||||
filterName = parser.text();
|
||||
} else {
|
||||
|
@ -68,23 +72,43 @@ public class ExistsFilterParser implements FilterParser {
|
|||
}
|
||||
}
|
||||
|
||||
if (fieldName == null) {
|
||||
if (fieldPattern == null) {
|
||||
throw new QueryParsingException(parseContext.index(), "exists must be provided with a [field]");
|
||||
}
|
||||
|
||||
Filter filter = null;
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null && smartNameFieldMappers.hasMapper()) {
|
||||
filter = smartNameFieldMappers.mapper().rangeFilter(null, null, true, true, parseContext);
|
||||
}
|
||||
if (filter == null) {
|
||||
filter = new TermRangeFilter(fieldName, null, null, true, true);
|
||||
MapperService.SmartNameObjectMapper smartNameObjectMapper = parseContext.smartObjectMapper(fieldPattern);
|
||||
if (smartNameObjectMapper != null && smartNameObjectMapper.hasMapper()) {
|
||||
// automatic make the object mapper pattern
|
||||
fieldPattern = fieldPattern + ".*";
|
||||
}
|
||||
|
||||
// we always cache this one, really does not change...
|
||||
filter = parseContext.cacheFilter(filter, null);
|
||||
Set<String> fields = parseContext.simpleMatchToIndexNames(fieldPattern);
|
||||
if (fields.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
MapperService.SmartNameFieldMappers nonNullFieldMappers = null;
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
XBooleanFilter boolFilter = new XBooleanFilter();
|
||||
for (String field : fields) {
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(field);
|
||||
if (smartNameFieldMappers != null) {
|
||||
nonNullFieldMappers = smartNameFieldMappers;
|
||||
}
|
||||
Filter filter = null;
|
||||
if (smartNameFieldMappers != null && smartNameFieldMappers.hasMapper()) {
|
||||
filter = smartNameFieldMappers.mapper().rangeFilter(null, null, true, true, parseContext);
|
||||
}
|
||||
if (filter == null) {
|
||||
filter = new TermRangeFilter(field, null, null, true, true);
|
||||
}
|
||||
boolFilter.add(filter, BooleanClause.Occur.SHOULD);
|
||||
}
|
||||
|
||||
// we always cache this one, really does not change... (exists)
|
||||
// its ok to cache under the fieldName cacheKey, since its per segment and the mapping applies to this data on this segment...
|
||||
Filter filter = parseContext.cacheFilter(boolFilter, new CacheKeyFilter.Key("$exists$" + fieldPattern));
|
||||
|
||||
filter = wrapSmartNameFilter(filter, nonNullFieldMappers, parseContext);
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, filter);
|
||||
}
|
||||
|
|
|
@ -26,9 +26,11 @@ import org.elasticsearch.common.inject.Inject;
|
|||
import org.elasticsearch.common.lucene.search.NotFilter;
|
||||
import org.elasticsearch.common.lucene.search.XBooleanFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.cache.filter.support.CacheKeyFilter;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.elasticsearch.index.query.support.QueryParsers.wrapSmartNameFilter;
|
||||
|
||||
|
@ -52,7 +54,7 @@ public class MissingFilterParser implements FilterParser {
|
|||
public Filter parse(QueryParseContext parseContext) throws IOException, QueryParsingException {
|
||||
XContentParser parser = parseContext.parser();
|
||||
|
||||
String fieldName = null;
|
||||
String fieldPattern = null;
|
||||
String filterName = null;
|
||||
boolean nullValue = false;
|
||||
boolean existence = true;
|
||||
|
@ -64,7 +66,7 @@ public class MissingFilterParser implements FilterParser {
|
|||
currentFieldName = parser.currentName();
|
||||
} else if (token.isValue()) {
|
||||
if ("field".equals(currentFieldName)) {
|
||||
fieldName = parser.text();
|
||||
fieldPattern = parser.text();
|
||||
} else if ("null_value".equals(currentFieldName)) {
|
||||
nullValue = parser.booleanValue();
|
||||
} else if ("existence".equals(currentFieldName)) {
|
||||
|
@ -77,7 +79,7 @@ public class MissingFilterParser implements FilterParser {
|
|||
}
|
||||
}
|
||||
|
||||
if (fieldName == null) {
|
||||
if (fieldPattern == null) {
|
||||
throw new QueryParsingException(parseContext.index(), "missing must be provided with a [field]");
|
||||
}
|
||||
|
||||
|
@ -85,33 +87,56 @@ public class MissingFilterParser implements FilterParser {
|
|||
throw new QueryParsingException(parseContext.index(), "missing must have either existence, or null_value, or both set to true");
|
||||
}
|
||||
|
||||
MapperService.SmartNameObjectMapper smartNameObjectMapper = parseContext.smartObjectMapper(fieldPattern);
|
||||
if (smartNameObjectMapper != null && smartNameObjectMapper.hasMapper()) {
|
||||
// automatic make the object mapper pattern
|
||||
fieldPattern = fieldPattern + ".*";
|
||||
}
|
||||
|
||||
Set<String> fields = parseContext.simpleMatchToIndexNames(fieldPattern);
|
||||
if (fields.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Filter existenceFilter = null;
|
||||
Filter nullFilter = null;
|
||||
|
||||
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
MapperService.SmartNameFieldMappers nonNullFieldMappers = null;
|
||||
|
||||
if (existence) {
|
||||
if (smartNameFieldMappers != null && smartNameFieldMappers.hasMapper()) {
|
||||
existenceFilter = smartNameFieldMappers.mapper().rangeFilter(null, null, true, true, parseContext);
|
||||
}
|
||||
if (existenceFilter == null) {
|
||||
existenceFilter = new TermRangeFilter(fieldName, null, null, true, true);
|
||||
XBooleanFilter boolFilter = new XBooleanFilter();
|
||||
for (String field : fields) {
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(field);
|
||||
if (smartNameFieldMappers != null) {
|
||||
nonNullFieldMappers = smartNameFieldMappers;
|
||||
}
|
||||
Filter filter = null;
|
||||
if (smartNameFieldMappers != null && smartNameFieldMappers.hasMapper()) {
|
||||
filter = smartNameFieldMappers.mapper().rangeFilter(null, null, true, true, parseContext);
|
||||
}
|
||||
if (filter == null) {
|
||||
filter = new TermRangeFilter(field, null, null, true, true);
|
||||
}
|
||||
boolFilter.add(filter, BooleanClause.Occur.SHOULD);
|
||||
}
|
||||
|
||||
// we always cache this one, really does not change... (exists)
|
||||
existenceFilter = parseContext.cacheFilter(existenceFilter, null);
|
||||
// its ok to cache under the fieldName cacheKey, since its per segment and the mapping applies to this data on this segment...
|
||||
existenceFilter = parseContext.cacheFilter(boolFilter, new CacheKeyFilter.Key("$exists$" + fieldPattern));
|
||||
existenceFilter = new NotFilter(existenceFilter);
|
||||
// cache the not filter as well, so it will be faster
|
||||
existenceFilter = parseContext.cacheFilter(existenceFilter, null);
|
||||
existenceFilter = parseContext.cacheFilter(existenceFilter, new CacheKeyFilter.Key("$missing$" + fieldPattern));
|
||||
}
|
||||
|
||||
if (nullValue) {
|
||||
if (smartNameFieldMappers != null && smartNameFieldMappers.hasMapper()) {
|
||||
nullFilter = smartNameFieldMappers.mapper().nullValueFilter();
|
||||
if (nullFilter != null) {
|
||||
// cache the not filter as well, so it will be faster
|
||||
nullFilter = parseContext.cacheFilter(nullFilter, null);
|
||||
for (String field : fields) {
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(field);
|
||||
if (smartNameFieldMappers != null && smartNameFieldMappers.hasMapper()) {
|
||||
nullFilter = smartNameFieldMappers.mapper().nullValueFilter();
|
||||
if (nullFilter != null) {
|
||||
// cache the not filter as well, so it will be faster
|
||||
nullFilter = parseContext.cacheFilter(nullFilter, new CacheKeyFilter.Key("$null$" + fieldPattern));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -135,7 +160,7 @@ public class MissingFilterParser implements FilterParser {
|
|||
return null;
|
||||
}
|
||||
|
||||
filter = wrapSmartNameFilter(filter, smartNameFieldMappers, parseContext);
|
||||
filter = wrapSmartNameFilter(filter, nonNullFieldMappers, parseContext);
|
||||
if (filterName != null) {
|
||||
parseContext.addNamedFilter(filterName, existenceFilter);
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*
|
||||
|
@ -265,6 +266,10 @@ public class QueryParseContext {
|
|||
return smartMapper.names().indexName();
|
||||
}
|
||||
|
||||
public Set<String> simpleMatchToIndexNames(String pattern) {
|
||||
return indexQueryParser.mapperService.simpleMatchToIndexNames(pattern, getTypes());
|
||||
}
|
||||
|
||||
public MapperService.SmartNameFieldMappers smartFieldMappers(String name) {
|
||||
return indexQueryParser.mapperService.smartName(name, getTypes());
|
||||
}
|
||||
|
|
|
@ -19,25 +19,26 @@
|
|||
|
||||
package org.elasticsearch.test.integration.search.query;
|
||||
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.query.*;
|
||||
import org.elasticsearch.index.query.CommonTermsQueryBuilder.Operator;
|
||||
import org.elasticsearch.index.query.MatchQueryBuilder.Type;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.search.facet.FacetBuilders;
|
||||
import org.elasticsearch.test.integration.AbstractSharedClusterTest;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.existsFilter;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.idsFilter;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.limitFilter;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.missingFilter;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.termFilter;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.termsFilter;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.termsLookupFilter;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.typeFilter;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.filteredQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.idsQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.queryString;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.termsQuery;
|
||||
import static org.elasticsearch.index.query.FilterBuilders.*;
|
||||
import static org.elasticsearch.index.query.QueryBuilders.*;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
@ -46,40 +47,16 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
import static org.testng.Assert.assertTrue;
|
||||
import static org.testng.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.common.Priority;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.query.BoolQueryBuilder;
|
||||
import org.elasticsearch.index.query.CommonTermsQueryBuilder.Operator;
|
||||
import org.elasticsearch.index.query.FilterBuilders;
|
||||
import org.elasticsearch.index.query.MatchQueryBuilder;
|
||||
import org.elasticsearch.index.query.MatchQueryBuilder.Type;
|
||||
import org.elasticsearch.index.query.MultiMatchQueryBuilder;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.index.query.TermQueryBuilder;
|
||||
import org.elasticsearch.index.query.WrapperFilterBuilder;
|
||||
import org.elasticsearch.index.query.WrapperQueryBuilder;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.search.facet.FacetBuilders;
|
||||
import org.elasticsearch.test.integration.AbstractSharedClusterTest;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class SimpleQueryTests extends AbstractSharedClusterTest {
|
||||
|
||||
|
||||
|
||||
public int numberOfNodes() {
|
||||
return 4;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void passQueryAsStringTest() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet();
|
||||
|
@ -344,10 +321,10 @@ public class SimpleQueryTests extends AbstractSharedClusterTest {
|
|||
public void filterExistsMissingTests() throws Exception {
|
||||
client().admin().indices().prepareCreate("test").setSettings(ImmutableSettings.settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet();
|
||||
|
||||
client().prepareIndex("test", "type1", "1").setSource("field1", "value1_1", "field2", "value2_1").execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "2").setSource("field1", "value1_2").execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "3").setSource("field2", "value2_3").execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "4").setSource("field3", "value3_4").execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject().startObject("obj1").field("obj1_val", "1").endObject().field("x1", "x_1").field("field1", "value1_1").field("field2", "value2_1").endObject()).execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject().startObject("obj1").field("obj1_val", "1").endObject().field("x2", "x_2").field("field1", "value1_2").endObject()).execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "3").setSource(jsonBuilder().startObject().startObject("obj2").field("obj2_val", "1").endObject().field("y1", "y_1").field("field2", "value2_3").endObject()).execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "4").setSource(jsonBuilder().startObject().startObject("obj2").field("obj2_val", "1").endObject().field("y2", "y_2").field("field3", "value3_4").endObject()).execute().actionGet();
|
||||
|
||||
client().admin().indices().prepareRefresh().execute().actionGet();
|
||||
|
||||
|
@ -375,12 +352,24 @@ public class SimpleQueryTests extends AbstractSharedClusterTest {
|
|||
assertThat(searchResponse.getHits().totalHits(), equalTo(1l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), equalTo("4"));
|
||||
|
||||
// wildcard check
|
||||
searchResponse = client().prepareSearch().setQuery(filteredQuery(matchAllQuery(), existsFilter("x*"))).execute().actionGet();
|
||||
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("1"), equalTo("2")));
|
||||
assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("1"), equalTo("2")));
|
||||
|
||||
// object check
|
||||
searchResponse = client().prepareSearch().setQuery(filteredQuery(matchAllQuery(), existsFilter("obj1"))).execute().actionGet();
|
||||
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("1"), equalTo("2")));
|
||||
assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("1"), equalTo("2")));
|
||||
|
||||
|
||||
searchResponse = client().prepareSearch().setQuery(filteredQuery(matchAllQuery(), missingFilter("field1"))).execute().actionGet();
|
||||
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
|
||||
// double check for cache
|
||||
searchResponse = client().prepareSearch().setQuery(filteredQuery(matchAllQuery(), missingFilter("field1"))).execute().actionGet();
|
||||
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
|
@ -395,6 +384,18 @@ public class SimpleQueryTests extends AbstractSharedClusterTest {
|
|||
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
|
||||
// wildcard check
|
||||
searchResponse = client().prepareSearch().setQuery(filteredQuery(matchAllQuery(), missingFilter("x*"))).execute().actionGet();
|
||||
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
|
||||
// object check
|
||||
searchResponse = client().prepareSearch().setQuery(filteredQuery(matchAllQuery(), missingFilter("obj1"))).execute().actionGet();
|
||||
assertThat(searchResponse.getHits().totalHits(), equalTo(2l));
|
||||
assertThat(searchResponse.getHits().getAt(0).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
assertThat(searchResponse.getHits().getAt(1).id(), anyOf(equalTo("3"), equalTo("4")));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -745,7 +746,7 @@ public class SimpleQueryTests extends AbstractSharedClusterTest {
|
|||
|
||||
@Test
|
||||
public void testEmptyTermsFilter() throws Exception {
|
||||
createIndexMapped("test", "type", "term", "string");
|
||||
createIndexMapped("test", "type", "term", "string");
|
||||
ensureGreen();
|
||||
client().prepareIndex("test", "type", "1").setSource("term", "1").execute().actionGet();
|
||||
client().prepareIndex("test", "type", "2").setSource("term", "2").execute().actionGet();
|
||||
|
@ -799,7 +800,7 @@ public class SimpleQueryTests extends AbstractSharedClusterTest {
|
|||
client().prepareIndex("test", "type", "3").setSource("term", "3").execute().actionGet();
|
||||
client().prepareIndex("test", "type", "4").setSource("term", "4").execute().actionGet();
|
||||
refresh();
|
||||
|
||||
|
||||
SearchResponse searchResponse = client().prepareSearch("test")
|
||||
.setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms"))
|
||||
).execute().actionGet();
|
||||
|
|
Loading…
Reference in New Issue