Internal: count api to become a shortcut to the search api

The count api used to have its own execution path, although it would do the same (up to bugs!) of the search api. This commit makes it a shortcut to the search api with size set to 0. The change is made in a backwards compatible manner, by leaving all of the java api code around too, given that you may not want to get back a whole SearchResponse when asking only for number of hits matching a query, also cause migrating from countResponse.getCount() to searchResponse.getHits().totalHits() doesn't look great from a user perspective. We can always decide to drop more code around the count api if we want to break backwards compatibility on the java api, making it a shortcut on the rest layer only.

Closes #9117
Closes #11198
This commit is contained in:
javanna 2015-05-16 12:58:27 +02:00 committed by Luca Cavanna
parent 1fa21a76cf
commit 6c81a8daf3
19 changed files with 263 additions and 1353 deletions

View File

@ -404,6 +404,12 @@ The `count` search type has been deprecated. All benefits from this search type
now be achieved by using the `query_then_fetch` search type (which is the
default) and setting `size` to `0`.
=== The count api internally uses the search api
The count api is now a shortcut to the search api with `size` set to 0. As a
result, a total failure will result in an exception being returned rather
than a normal response with `count` set to `0` and shard failures.
=== JSONP support
JSONP callback support has now been removed. CORS should be used to access Elasticsearch

View File

@ -122,8 +122,6 @@ import org.elasticsearch.action.admin.indices.warmer.put.TransportPutWarmerActio
import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.TransportBulkAction;
import org.elasticsearch.action.bulk.TransportShardBulkAction;
import org.elasticsearch.action.count.CountAction;
import org.elasticsearch.action.count.TransportCountAction;
import org.elasticsearch.action.delete.DeleteAction;
import org.elasticsearch.action.delete.TransportDeleteAction;
import org.elasticsearch.action.exists.ExistsAction;
@ -273,7 +271,6 @@ public class ActionModule extends AbstractModule {
registerAction(MultiTermVectorsAction.INSTANCE, TransportMultiTermVectorsAction.class,
TransportShardMultiTermsVectorAction.class);
registerAction(DeleteAction.INSTANCE, TransportDeleteAction.class);
registerAction(CountAction.INSTANCE, TransportCountAction.class);
registerAction(ExistsAction.INSTANCE, TransportExistsAction.class);
registerAction(SuggestAction.INSTANCE, TransportSuggestAction.class);
registerAction(UpdateAction.INSTANCE, TransportUpdateAction.class);

View File

@ -20,15 +20,9 @@
package org.elasticsearch.action;
import com.google.common.base.Preconditions;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.support.PlainListenableActionFuture;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.ClusterAdminClient;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.client.IndicesAdminClient;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilder;
import org.elasticsearch.threadpool.ThreadPool;
/**
@ -87,7 +81,7 @@ public abstract class ActionRequestBuilder<Request extends ActionRequest, Respon
return execute().actionGet(timeout);
}
public final void execute(ActionListener<Response> listener) {
public void execute(ActionListener<Response> listener) {
client.execute(action, beforeExecute(request), listener);
}

View File

@ -23,6 +23,8 @@ import org.elasticsearch.action.Action;
import org.elasticsearch.client.ElasticsearchClient;
/**
* Action that shortcuts to the search api with size set to 0. It doesn't have a corresponding
* transport action, it just runs the search api internally.
*/
public class CountAction extends Action<CountRequest, CountResponse, CountRequestBuilder> {
@ -35,7 +37,7 @@ public class CountAction extends Action<CountRequest, CountResponse, CountReques
@Override
public CountResponse newResponse() {
return new CountResponse();
throw new UnsupportedOperationException("CountAction doesn't have its own transport action, gets executed as a SearchAction internally");
}
@Override

View File

@ -20,7 +20,7 @@
package org.elasticsearch.action.count;
import org.elasticsearch.ElasticsearchGenerationException;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.QuerySourceBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest;
import org.elasticsearch.client.Requests;
@ -33,6 +33,7 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import java.io.IOException;
import java.util.Arrays;
@ -67,12 +68,8 @@ public class CountRequest extends BroadcastOperationRequest<CountRequest> {
private String[] types = Strings.EMPTY_ARRAY;
long nowInMillis;
private int terminateAfter = DEFAULT_TERMINATE_AFTER;
CountRequest() {
}
/**
* Constructs a new count request against the provided indices. No indices provided means it will
* run against all indices.
@ -81,12 +78,6 @@ public class CountRequest extends BroadcastOperationRequest<CountRequest> {
super(indices);
}
@Override
public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = super.validate();
return validationException;
}
/**
* The minimum score of the documents to include in the count.
*/
@ -121,6 +112,7 @@ public class CountRequest extends BroadcastOperationRequest<CountRequest> {
/**
* The source to execute in the form of a map.
*/
@SuppressWarnings("unchecked")
public CountRequest source(Map querySource) {
try {
XContentBuilder builder = XContentFactory.contentBuilder(Requests.CONTENT_TYPE);
@ -228,24 +220,12 @@ public class CountRequest extends BroadcastOperationRequest<CountRequest> {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
minScore = in.readFloat();
routing = in.readOptionalString();
preference = in.readOptionalString();
source = in.readBytesReference();
types = in.readStringArray();
terminateAfter = in.readVInt();
throw new UnsupportedOperationException("CountRequest doesn't support being sent over the wire, just a shortcut to the search api");
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeFloat(minScore);
out.writeOptionalString(routing);
out.writeOptionalString(preference);
out.writeBytesReference(source);
out.writeStringArray(types);
out.writeVInt(terminateAfter);
throw new UnsupportedOperationException("CountRequest doesn't support being sent over the wire, just a shortcut to the search api");
}
@Override
@ -258,4 +238,23 @@ public class CountRequest extends BroadcastOperationRequest<CountRequest> {
}
return "[" + Arrays.toString(indices) + "]" + Arrays.toString(types) + ", source[" + sSource + "]";
}
public SearchRequest toSearchRequest() {
SearchRequest searchRequest = new SearchRequest(indices());
searchRequest.indicesOptions(indicesOptions());
searchRequest.types(types());
searchRequest.routing(routing());
searchRequest.preference(preference());
searchRequest.source(source());
SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder();
searchSourceBuilder.size(0);
if (minScore() != DEFAULT_MIN_SCORE) {
searchSourceBuilder.minScore(minScore());
}
if (terminateAfter() != DEFAULT_TERMINATE_AFTER) {
searchSourceBuilder.terminateAfter(terminateAfter());
}
searchRequest.extraSource(searchSourceBuilder);
return searchRequest;
}
}

View File

@ -20,6 +20,10 @@
package org.elasticsearch.action.count;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.DelegatingActionListener;
import org.elasticsearch.action.support.QuerySourceBuilder;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
@ -145,6 +149,17 @@ public class CountRequestBuilder extends BroadcastOperationRequestBuilder<CountR
return sourceBuilder;
}
@Override
public void execute(ActionListener<CountResponse> listener) {
CountRequest countRequest = beforeExecute(request);
client.execute(SearchAction.INSTANCE, countRequest.toSearchRequest(), new DelegatingActionListener<SearchResponse, CountResponse>(listener) {
@Override
protected CountResponse getDelegatedFromInstigator(SearchResponse response) {
return new CountResponse(response);
}
});
}
@Override
public String toString() {
if (sourceBuilder != null) {

View File

@ -19,30 +19,27 @@
package org.elasticsearch.action.count;
import java.io.IOException;
import java.util.List;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
import java.util.Arrays;
/**
* The response of the count action.
*/
public class CountResponse extends BroadcastOperationResponse {
private boolean terminatedEarly;
private long count;
private final boolean terminatedEarly;
private final long count;
CountResponse() {
}
CountResponse(long count, boolean hasTerminatedEarly, int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) {
super(totalShards, successfulShards, failedShards, shardFailures);
this.count = count;
this.terminatedEarly = hasTerminatedEarly;
public CountResponse(SearchResponse searchResponse) {
super(searchResponse.getTotalShards(), searchResponse.getSuccessfulShards(), searchResponse.getFailedShards(), Arrays.asList(searchResponse.getShardFailures()));
this.count = searchResponse.getHits().totalHits();
this.terminatedEarly = searchResponse.isTerminatedEarly() != null && searchResponse.isTerminatedEarly();
}
/**
@ -65,15 +62,11 @@ public class CountResponse extends BroadcastOperationResponse {
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
count = in.readVLong();
terminatedEarly = in.readBoolean();
throw new UnsupportedOperationException("CountResponse doesn't support being sent over the wire, just a shortcut to the search api");
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVLong(count);
out.writeBoolean(terminatedEarly);
throw new UnsupportedOperationException("CountResponse doesn't support being sent over the wire, just a shortcut to the search api");
}
}

View File

@ -1,137 +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.action.count;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationRequest;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TERMINATE_AFTER;
/**
* Internal count request executed directly against a specific index shard.
*/
class ShardCountRequest extends BroadcastShardOperationRequest {
private float minScore;
private int terminateAfter;
private BytesReference querySource;
private String[] types = Strings.EMPTY_ARRAY;
private long nowInMillis;
@Nullable
private String[] filteringAliases;
ShardCountRequest() {
}
ShardCountRequest(ShardId shardId, @Nullable String[] filteringAliases, CountRequest request) {
super(shardId, request);
this.minScore = request.minScore();
this.querySource = request.source();
this.types = request.types();
this.filteringAliases = filteringAliases;
this.nowInMillis = request.nowInMillis;
this.terminateAfter = request.terminateAfter();
}
public float minScore() {
return minScore;
}
public BytesReference querySource() {
return querySource;
}
public String[] types() {
return this.types;
}
public String[] filteringAliases() {
return filteringAliases;
}
public long nowInMillis() {
return this.nowInMillis;
}
public int terminateAfter() {
return this.terminateAfter;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
minScore = in.readFloat();
querySource = in.readBytesReference();
int typesSize = in.readVInt();
if (typesSize > 0) {
types = new String[typesSize];
for (int i = 0; i < typesSize; i++) {
types[i] = in.readString();
}
}
int aliasesSize = in.readVInt();
if (aliasesSize > 0) {
filteringAliases = new String[aliasesSize];
for (int i = 0; i < aliasesSize; i++) {
filteringAliases[i] = in.readString();
}
}
nowInMillis = in.readVLong();
terminateAfter = in.readVInt();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeFloat(minScore);
out.writeBytesReference(querySource);
out.writeVInt(types.length);
for (String type : types) {
out.writeString(type);
}
if (filteringAliases != null) {
out.writeVInt(filteringAliases.length);
for (String alias : filteringAliases) {
out.writeString(alias);
}
} else {
out.writeVInt(0);
}
out.writeVLong(nowInMillis);
out.writeVInt(terminateAfter);
}
}

View File

@ -1,71 +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.action.count;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
/**
* Internal count response of a shard count request executed directly against a specific shard.
*
*
*/
class ShardCountResponse extends BroadcastShardOperationResponse {
private long count;
private boolean terminatedEarly;
ShardCountResponse() {
}
ShardCountResponse(ShardId shardId, long count, boolean terminatedEarly) {
super(shardId);
this.count = count;
this.terminatedEarly = terminatedEarly;
}
public long getCount() {
return this.count;
}
public boolean terminatedEarly() {
return this.terminatedEarly;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
count = in.readVLong();
terminatedEarly = in.readBoolean();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVLong(count);
out.writeBoolean(terminatedEarly);
}
}

View File

@ -1,193 +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.action.count;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.TransportBroadcastOperationAction;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.internal.DefaultSearchContext;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.ShardSearchLocalRequest;
import org.elasticsearch.search.query.QueryPhaseExecutionException;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReferenceArray;
import static com.google.common.collect.Lists.newArrayList;
import static org.elasticsearch.search.internal.SearchContext.DEFAULT_TERMINATE_AFTER;
/**
*
*/
public class TransportCountAction extends TransportBroadcastOperationAction<CountRequest, CountResponse, ShardCountRequest, ShardCountResponse> {
private final IndicesService indicesService;
private final ScriptService scriptService;
private final PageCacheRecycler pageCacheRecycler;
private final BigArrays bigArrays;
@Inject
public TransportCountAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService,
IndicesService indicesService, ScriptService scriptService, PageCacheRecycler pageCacheRecycler,
BigArrays bigArrays, ActionFilters actionFilters) {
super(settings, CountAction.NAME, threadPool, clusterService, transportService, actionFilters,
CountRequest.class, ShardCountRequest.class, ThreadPool.Names.SEARCH);
this.indicesService = indicesService;
this.scriptService = scriptService;
this.pageCacheRecycler = pageCacheRecycler;
this.bigArrays = bigArrays;
}
@Override
protected void doExecute(CountRequest request, ActionListener<CountResponse> listener) {
request.nowInMillis = System.currentTimeMillis();
super.doExecute(request, listener);
}
@Override
protected ShardCountRequest newShardRequest(int numShards, ShardRouting shard, CountRequest request) {
String[] filteringAliases = clusterService.state().metaData().filteringAliases(shard.index(), request.indices());
return new ShardCountRequest(shard.shardId(), filteringAliases, request);
}
@Override
protected ShardCountResponse newShardResponse() {
return new ShardCountResponse();
}
@Override
protected GroupShardsIterator shards(ClusterState clusterState, CountRequest request, String[] concreteIndices) {
Map<String, Set<String>> routingMap = clusterState.metaData().resolveSearchRouting(request.routing(), request.indices());
return clusterService.operationRouting().searchShards(clusterState, request.indices(), concreteIndices, routingMap, request.preference());
}
@Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, CountRequest request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.READ);
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, CountRequest countRequest, String[] concreteIndices) {
return state.blocks().indicesBlockedException(ClusterBlockLevel.READ, concreteIndices);
}
@Override
protected CountResponse newResponse(CountRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) {
int successfulShards = 0;
int failedShards = 0;
long count = 0;
boolean terminatedEarly = false;
List<ShardOperationFailedException> shardFailures = null;
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
// simply ignore non active shards
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = newArrayList();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
count += ((ShardCountResponse) shardResponse).getCount();
if (((ShardCountResponse) shardResponse).terminatedEarly()) {
terminatedEarly = true;
}
successfulShards++;
}
}
return new CountResponse(count, terminatedEarly, shardsResponses.length(), successfulShards, failedShards, shardFailures);
}
@Override
protected ShardCountResponse shardOperation(ShardCountRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id());
SearchShardTarget shardTarget = new SearchShardTarget(clusterService.localNode().id(), request.shardId().getIndex(), request.shardId().id());
SearchContext context = new DefaultSearchContext(0,
new ShardSearchLocalRequest(request.types(), request.nowInMillis(), request.filteringAliases()),
shardTarget, indexShard.acquireSearcher("count"), indexService, indexShard,
scriptService, pageCacheRecycler, bigArrays, threadPool.estimatedTimeInMillisCounter());
SearchContext.setCurrent(context);
try {
// TODO: min score should move to be "null" as a value that is not initialized...
if (request.minScore() != -1) {
context.minimumScore(request.minScore());
}
BytesReference source = request.querySource();
if (source != null && source.length() > 0) {
try {
QueryParseContext.setTypes(request.types());
context.parsedQuery(indexService.queryParserService().parseQuery(source));
} finally {
QueryParseContext.removeTypes();
}
}
final boolean hasTerminateAfterCount = request.terminateAfter() != DEFAULT_TERMINATE_AFTER;
boolean terminatedEarly = false;
context.preProcess();
try {
long count;
if (hasTerminateAfterCount) {
final Lucene.EarlyTerminatingCollector countCollector =
Lucene.createCountBasedEarlyTerminatingCollector(request.terminateAfter());
terminatedEarly = Lucene.countWithEarlyTermination(context.searcher(), context.query(), countCollector);
count = countCollector.count();
} else {
count = Lucene.count(context.searcher(), context.query());
}
return new ShardCountResponse(request.shardId(), count, terminatedEarly);
} catch (Exception e) {
throw new QueryPhaseExecutionException(context, "failed to execute count", e);
}
} finally {
// this will also release the index searcher
context.close();
SearchContext.removeCurrent();
}
}
}

View File

@ -19,16 +19,16 @@
package org.elasticsearch.action.support.broadcast;
import static org.elasticsearch.action.support.DefaultShardOperationFailedException.readShardOperationFailed;
import java.io.IOException;
import java.util.List;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
import java.util.List;
import static org.elasticsearch.action.support.DefaultShardOperationFailedException.readShardOperationFailed;
/**
* Base class for all broadcast operation based responses.
*/
@ -42,7 +42,7 @@ public abstract class BroadcastOperationResponse extends ActionResponse {
protected BroadcastOperationResponse() {
}
protected BroadcastOperationResponse(int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) {
protected BroadcastOperationResponse(int totalShards, int successfulShards, int failedShards, List<? extends ShardOperationFailedException> shardFailures) {
this.totalShards = totalShards;
this.successfulShards = successfulShards;
this.failedShards = failedShards;

View File

@ -259,6 +259,8 @@ import org.elasticsearch.action.suggest.SuggestAction;
import org.elasticsearch.action.suggest.SuggestRequest;
import org.elasticsearch.action.suggest.SuggestRequestBuilder;
import org.elasticsearch.action.suggest.SuggestResponse;
import org.elasticsearch.action.support.AdapterActionFuture;
import org.elasticsearch.action.support.DelegatingActionListener;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.ThreadedActionListener;
import org.elasticsearch.action.termvectors.*;
@ -335,7 +337,6 @@ public abstract class AbstractClient extends AbstractComponent implements Client
protected abstract <Request extends ActionRequest, Response extends ActionResponse, RequestBuilder extends ActionRequestBuilder<Request, Response, RequestBuilder>> void doExecute(final Action<Request, Response, RequestBuilder> action, final Request request, ActionListener<Response> listener);
@Override
public ActionFuture<IndexResponse> index(final IndexRequest request) {
return execute(IndexAction.INSTANCE, request);
@ -594,12 +595,24 @@ public abstract class AbstractClient extends AbstractComponent implements Client
@Override
public ActionFuture<CountResponse> count(final CountRequest request) {
return execute(CountAction.INSTANCE, request);
AdapterActionFuture<CountResponse, SearchResponse> actionFuture = new AdapterActionFuture<CountResponse, SearchResponse>() {
@Override
protected CountResponse convert(SearchResponse listenerResponse) {
return new CountResponse(listenerResponse);
}
};
execute(SearchAction.INSTANCE, request.toSearchRequest(), actionFuture);
return actionFuture;
}
@Override
public void count(final CountRequest request, final ActionListener<CountResponse> listener) {
execute(CountAction.INSTANCE, request, listener);
execute(SearchAction.INSTANCE, request.toSearchRequest(), new DelegatingActionListener<SearchResponse, CountResponse>(listener) {
@Override
protected CountResponse getDelegatedFromInstigator(SearchResponse response) {
return new CountResponse(response);
}
});
}
@Override

View File

@ -47,10 +47,7 @@ import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.search.suggest.SuggestBuilder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.*;
/**
* A search source builder allowing to easily build search source. Simple
@ -188,6 +185,7 @@ public class SearchSourceBuilder extends ToXContentToBytes {
/**
* Constructs a new search source builder with a query from a map.
*/
@SuppressWarnings("unchecked")
public SearchSourceBuilder query(Map query) {
try {
XContentBuilder builder = XContentFactory.contentBuilder(Requests.CONTENT_TYPE);
@ -251,6 +249,7 @@ public class SearchSourceBuilder extends ToXContentToBytes {
/**
* Constructs a new search source builder with a query from a map.
*/
@SuppressWarnings("unchecked")
public SearchSourceBuilder postFilter(Map postFilter) {
try {
XContentBuilder builder = XContentFactory.contentBuilder(Requests.CONTENT_TYPE);
@ -415,9 +414,6 @@ public class SearchSourceBuilder extends ToXContentToBytes {
/**
* Set the rescore window size for rescores that don't specify their window.
*
* @param defaultRescoreWindowSize
* @return
*/
public SearchSourceBuilder defaultRescoreWindowSize(int defaultRescoreWindowSize) {
this.defaultRescoreWindowSize = defaultRescoreWindowSize;
@ -427,6 +423,7 @@ public class SearchSourceBuilder extends ToXContentToBytes {
/**
* Sets a raw (xcontent / json) addAggregation.
*/
@SuppressWarnings("unchecked")
public SearchSourceBuilder aggregations(Map aggregations) {
try {
XContentBuilder builder = XContentFactory.contentBuilder(Requests.CONTENT_TYPE);
@ -482,9 +479,6 @@ public class SearchSourceBuilder extends ToXContentToBytes {
/**
* Indicates whether the response should contain the stored _source for
* every hit
*
* @param fetch
* @return
*/
public SearchSourceBuilder fetchSource(boolean fetch) {
if (this.fetchSourceContext == null) {
@ -563,9 +557,7 @@ public class SearchSourceBuilder extends ToXContentToBytes {
if (fieldNames == null) {
fieldNames = new ArrayList<>();
}
for (String field : fields) {
fieldNames.add(field);
}
Collections.addAll(fieldNames, fields);
return this;
}
@ -777,7 +769,7 @@ public class SearchSourceBuilder extends ToXContentToBytes {
}
if (trackScores) {
builder.field("track_scores", trackScores);
builder.field("track_scores", true);
}
if (indexBoost != null) {

View File

@ -57,8 +57,6 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryRequest;
import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.count.CountAction;
import org.elasticsearch.action.count.CountRequest;
import org.elasticsearch.action.delete.DeleteAction;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.exists.ExistsAction;
@ -356,18 +354,6 @@ public class IndicesRequestTests extends ElasticsearchIntegrationTest {
assertIndicesSubset(indices, multiGetShardAction);
}
@Test
public void testCount() {
String countShardAction = CountAction.NAME + "[s]";
interceptTransportActions(countShardAction);
CountRequest countRequest = new CountRequest(randomIndicesOrAliases());
internalCluster().clientNodeClient().count(countRequest).actionGet();
clearInterceptedActions();
assertSameIndices(countRequest, countShardAction);
}
@Test
public void testExists() {
String existsShardAction = ExistsAction.NAME + "[s]";

View File

@ -0,0 +1,110 @@
/*
* 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.action.count;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.QuerySourceBuilder;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test;
import java.util.Map;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.notNullValue;
import static org.hamcrest.CoreMatchers.nullValue;
public class CountRequestTests extends ElasticsearchTestCase {
@Test
public void testToSearchRequest() {
CountRequest countRequest;
if (randomBoolean()) {
countRequest = new CountRequest(randomStringArray());
} else {
countRequest = new CountRequest();
}
if (randomBoolean()) {
countRequest.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean()));
}
if (randomBoolean()) {
countRequest.types(randomStringArray());
}
if (randomBoolean()) {
countRequest.routing(randomStringArray());
}
if (randomBoolean()) {
countRequest.preference(randomAsciiOfLengthBetween(1, 10));
}
if (randomBoolean()) {
countRequest.source(new QuerySourceBuilder().setQuery(QueryBuilders.termQuery("field", "value")));
}
if (randomBoolean()) {
countRequest.minScore(randomFloat());
}
if (randomBoolean()) {
countRequest.terminateAfter(randomIntBetween(1, 1000));
}
SearchRequest searchRequest = countRequest.toSearchRequest();
assertThat(searchRequest.indices(), equalTo(countRequest.indices()));
assertThat(searchRequest.indicesOptions(), equalTo(countRequest.indicesOptions()));
assertThat(searchRequest.types(), equalTo(countRequest.types()));
assertThat(searchRequest.routing(), equalTo(countRequest.routing()));
assertThat(searchRequest.preference(), equalTo(countRequest.preference()));
if (countRequest.source() == null) {
assertThat(searchRequest.source(), nullValue());
} else {
Map<String, Object> sourceMap = XContentHelper.convertToMap(searchRequest.source(), false).v2();
assertThat(sourceMap.size(), equalTo(1));
assertThat(sourceMap.get("query"), notNullValue());
}
Map<String, Object> extraSourceMap = XContentHelper.convertToMap(searchRequest.extraSource(), false).v2();
int count = 1;
assertThat((Integer)extraSourceMap.get("size"), equalTo(0));
if (countRequest.minScore() == CountRequest.DEFAULT_MIN_SCORE) {
assertThat(extraSourceMap.get("min_score"), nullValue());
} else {
assertThat(((Number)extraSourceMap.get("min_score")).floatValue(), equalTo(countRequest.minScore()));
count++;
}
if (countRequest.terminateAfter() == SearchContext.DEFAULT_TERMINATE_AFTER) {
assertThat(extraSourceMap.get("terminate_after"), nullValue());
} else {
assertThat((Integer)extraSourceMap.get("terminate_after"), equalTo(countRequest.terminateAfter()));
count++;
}
assertThat(extraSourceMap.size(), equalTo(count));
}
private static String[] randomStringArray() {
int count = randomIntBetween(1, 5);
String[] indices = new String[count];
for (int i = 0; i < count; i++) {
indices[i] = randomAsciiOfLengthBetween(1, 10);
}
return indices;
}
}

View File

@ -0,0 +1,51 @@
/*
* 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.action.count;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.search.internal.InternalSearchHits;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test;
import static org.hamcrest.CoreMatchers.equalTo;
public class CountResponseTests extends ElasticsearchTestCase {
@Test
public void testFromSearchResponse() {
InternalSearchResponse internalSearchResponse = new InternalSearchResponse(new InternalSearchHits(null, randomLong(), randomFloat()), null, null, randomBoolean(), randomBoolean());
ShardSearchFailure[] shardSearchFailures = new ShardSearchFailure[randomIntBetween(0, 5)];
for (int i = 0; i < shardSearchFailures.length; i++) {
shardSearchFailures[i] = new ShardSearchFailure(new IllegalArgumentException());
}
SearchResponse searchResponse = new SearchResponse(internalSearchResponse, null, randomIntBetween(0, 100), randomIntBetween(0, 100), randomIntBetween(0, 100), shardSearchFailures);
CountResponse countResponse = new CountResponse(searchResponse);
assertThat(countResponse.getTotalShards(), equalTo(searchResponse.getTotalShards()));
assertThat(countResponse.getSuccessfulShards(), equalTo(searchResponse.getSuccessfulShards()));
assertThat(countResponse.getFailedShards(), equalTo(searchResponse.getFailedShards()));
assertThat(countResponse.getShardFailures(), equalTo((ShardOperationFailedException[])searchResponse.getShardFailures()));
assertThat(countResponse.getCount(), equalTo(searchResponse.getHits().totalHits()));
assertThat(countResponse.terminatedEarly(), equalTo(searchResponse.isTerminatedEarly()));
}
}

View File

@ -20,8 +20,8 @@
package org.elasticsearch.broadcast;
import com.google.common.base.Charsets;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
@ -33,7 +33,6 @@ import static org.elasticsearch.client.Requests.countRequest;
import static org.elasticsearch.client.Requests.indexRequest;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
public class BroadcastActionsTests extends ElasticsearchIntegrationTest {
@ -72,14 +71,10 @@ public class BroadcastActionsTests extends ElasticsearchIntegrationTest {
for (int i = 0; i < 5; i++) {
// test failed (simply query that can't be parsed)
CountResponse countResponse = client().count(countRequest("test").source("{ term : { _type : \"type1 } }".getBytes(Charsets.UTF_8))).actionGet();
assertThat(countResponse.getCount(), equalTo(0l));
assertThat(countResponse.getTotalShards(), equalTo(numShards.numPrimaries));
assertThat(countResponse.getSuccessfulShards(), equalTo(0));
assertThat(countResponse.getFailedShards(), equalTo(numShards.numPrimaries));
for (ShardOperationFailedException exp : countResponse.getShardFailures()) {
assertThat(exp.reason(), containsString("QueryParsingException"));
try {
client().count(countRequest("test").source("{ term : { _type : \"type1 } }".getBytes(Charsets.UTF_8))).actionGet();
} catch(SearchPhaseExecutionException e) {
assertThat(e.shardFailures().length, equalTo(numShards.numPrimaries));
}
}
}

View File

@ -1,843 +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.count.query;
import org.elasticsearch.Version;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.CommonTermsQueryBuilder.Operator;
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.WrapperQueryBuilder;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.joda.time.DateTime;
import org.joda.time.DateTimeZone;
import org.joda.time.format.ISODateTimeFormat;
import org.junit.Test;
import java.io.IOException;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
import static org.elasticsearch.index.query.QueryBuilders.existsQuery;
import static org.elasticsearch.index.query.QueryBuilders.filteredQuery;
import static org.elasticsearch.index.query.QueryBuilders.idsQuery;
import static org.elasticsearch.index.query.QueryBuilders.limitQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
import static org.elasticsearch.index.query.QueryBuilders.missingQuery;
import static org.elasticsearch.index.query.QueryBuilders.multiMatchQuery;
import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.index.query.QueryBuilders.termsLookupQuery;
import static org.elasticsearch.index.query.QueryBuilders.termsQuery;
import static org.elasticsearch.index.query.QueryBuilders.typeQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.equalTo;
public class CountQueryTests extends ElasticsearchIntegrationTest {
@Test
public void passQueryAsStringTest() throws Exception {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("field1", "value1_1", "field2", "value2_1").setRefresh(true).get();
CountResponse countResponse = client().prepareCount().setSource(new BytesArray("{ \"query\" : { \"term\" : { \"field1\" : \"value1_1\" }}}").array()).get();
assertHitCount(countResponse, 1l);
}
@Test
public void testIndexOptions() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("type1", "field1", "type=string,index_options=docs"));
client().prepareIndex("test", "type1", "1").setSource("field1", "quick brown fox", "field2", "quick brown fox").get();
client().prepareIndex("test", "type1", "2").setSource("field1", "quick lazy huge brown fox", "field2", "quick lazy huge brown fox").get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(QueryBuilders.matchQuery("field2", "quick brown").type(Type.PHRASE).slop(0)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.matchQuery("field1", "quick brown").type(Type.PHRASE).slop(0)).get();
assertHitCount(countResponse, 0l);
assertThat(countResponse.getFailedShards(), anyOf(equalTo(1), equalTo(2)));
assertThat(countResponse.getFailedShards(), equalTo(countResponse.getShardFailures().length));
for (ShardOperationFailedException shardFailure : countResponse.getShardFailures()) {
assertThat(shardFailure.status(), equalTo(RestStatus.INTERNAL_SERVER_ERROR));
assertThat(shardFailure.reason(), containsString("[field \"field1\" was indexed without position data; cannot run PhraseQuery"));
}
}
@Test
public void testCommonTermsQuery() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("type1", "field1", "type=string,analyzer=whitespace")
.setSettings(SETTING_NUMBER_OF_SHARDS, 1));
indexRandom(true,
client().prepareIndex("test", "type1", "3").setSource("field1", "quick lazy huge brown pidgin", "field2", "the quick lazy huge brown fox jumps over the tree"),
client().prepareIndex("test", "type1", "1").setSource("field1", "the quick brown fox"),
client().prepareIndex("test", "type1", "2").setSource("field1", "the quick lazy huge brown fox jumps over the tree") );
CountResponse countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the quick brown").cutoffFrequency(3).lowFreqOperator(Operator.OR)).get();
assertHitCount(countResponse, 3l);
countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the quick brown").cutoffFrequency(3).lowFreqOperator(Operator.AND)).get();
assertHitCount(countResponse, 2l);
// Default
countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the quick brown").cutoffFrequency(3)).get();
assertHitCount(countResponse, 3l);
countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the huge fox").lowFreqMinimumShouldMatch("2")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the lazy fox brown").cutoffFrequency(1).highFreqMinimumShouldMatch("3")).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the lazy fox brown").cutoffFrequency(1).highFreqMinimumShouldMatch("4")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setSource(new BytesArray("{ \"query\" : { \"common\" : { \"field1\" : { \"query\" : \"the lazy fox brown\", \"cutoff_frequency\" : 1, \"minimum_should_match\" : { \"high_freq\" : 4 } } } } }").array()).get();
assertHitCount(countResponse, 1l);
// Default
countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the lazy fox brown").cutoffFrequency(1)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.commonTermsQuery("field1", "the quick brown").cutoffFrequency(3).analyzer("standard")).get();
assertHitCount(countResponse, 3l);
// standard drops "the" since its a stopword
// try the same with match query
countResponse = client().prepareCount().setQuery(QueryBuilders.matchQuery("field1", "the quick brown").cutoffFrequency(3).operator(MatchQueryBuilder.Operator.AND)).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(QueryBuilders.matchQuery("field1", "the quick brown").cutoffFrequency(3).operator(MatchQueryBuilder.Operator.OR)).get();
assertHitCount(countResponse, 3l);
countResponse = client().prepareCount().setQuery(QueryBuilders.matchQuery("field1", "the quick brown").cutoffFrequency(3).operator(MatchQueryBuilder.Operator.AND).analyzer("stop")).get();
assertHitCount(countResponse, 3l);
// standard drops "the" since its a stopword
// try the same with multi match query
countResponse = client().prepareCount().setQuery(QueryBuilders.multiMatchQuery("the quick brown", "field1", "field2").cutoffFrequency(3).operator(MatchQueryBuilder.Operator.AND)).get();
assertHitCount(countResponse, 3l);
}
@Test
public void queryStringAnalyzedWildcard() throws Exception {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("field1", "value_1", "field2", "value_2").get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(queryStringQuery("value*").analyzeWildcard(true)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("*ue*").analyzeWildcard(true)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("*ue_1").analyzeWildcard(true)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("val*e_1").analyzeWildcard(true)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("v?l*e?1").analyzeWildcard(true)).get();
assertHitCount(countResponse, 1l);
}
@Test
public void testLowercaseExpandedTerms() {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("field1", "value_1", "field2", "value_2").get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(queryStringQuery("VALUE_3~1").lowercaseExpandedTerms(true)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("VALUE_3~1").lowercaseExpandedTerms(false)).get();
assertHitCount(countResponse, 0l);
countResponse = client().prepareCount().setQuery(queryStringQuery("ValUE_*").lowercaseExpandedTerms(true)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("vAl*E_1")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("[VALUE_1 TO VALUE_3]")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("[VALUE_1 TO VALUE_3]").lowercaseExpandedTerms(false)).get();
assertHitCount(countResponse, 0l);
}
@Test //https://github.com/elasticsearch/elasticsearch/issues/3540
public void testDateRangeInQueryString() {
//the mapping needs to be provided upfront otherwise we are not sure how many failures we get back
//as with dynamic mappings some shards might be lacking behind and parse a different query
assertAcked(prepareCreate("test").addMapping(
"type", "past", "type=date", "future", "type=date"
));
ensureGreen();
NumShards test = getNumShards("test");
String aMonthAgo = ISODateTimeFormat.yearMonthDay().print(new DateTime(DateTimeZone.UTC).minusMonths(1));
String aMonthFromNow = ISODateTimeFormat.yearMonthDay().print(new DateTime(DateTimeZone.UTC).plusMonths(1));
client().prepareIndex("test", "type", "1").setSource("past", aMonthAgo, "future", aMonthFromNow).get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(queryStringQuery("past:[now-2M/d TO now/d]")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("future:[now/d TO now+2M/d]").lowercaseExpandedTerms(false)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(queryStringQuery("future:[now/D TO now+2M/d]").lowercaseExpandedTerms(false)).get();
//D is an unsupported unit in date math
assertThat(countResponse.getSuccessfulShards(), equalTo(0));
assertThat(countResponse.getFailedShards(), equalTo(test.numPrimaries));
assertThat(countResponse.getShardFailures().length, equalTo(test.numPrimaries));
for (ShardOperationFailedException shardFailure : countResponse.getShardFailures()) {
assertThat(shardFailure.status(), equalTo(RestStatus.BAD_REQUEST));
assertThat(shardFailure.reason(), allOf(containsString("Failed to parse"), containsString("unit [D] not supported for date math")));
}
}
@Test
public void typeFilterTypeIndexedTests() throws Exception {
typeFilterTests("not_analyzed");
}
@Test
public void typeFilterTypeNotIndexedTests() throws Exception {
typeFilterTests("no");
}
private void typeFilterTests(String index) throws Exception {
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
assertAcked(prepareCreate("test").setSettings(indexSettings)
.addMapping("type1", jsonBuilder().startObject().startObject("type1")
.startObject("_type").field("index", index).endObject()
.endObject().endObject())
.addMapping("type2", jsonBuilder().startObject().startObject("type2")
.startObject("_type").field("index", index).endObject()
.endObject().endObject()));
indexRandom(true, client().prepareIndex("test", "type1", "1").setSource("field1", "value1"),
client().prepareIndex("test", "type2", "1").setSource("field1", "value1"),
client().prepareIndex("test", "type1", "2").setSource("field1", "value1"),
client().prepareIndex("test", "type2", "2").setSource("field1", "value1"),
client().prepareIndex("test", "type2", "3").setSource("field1", "value1"));
assertHitCount(client().prepareCount().setQuery(filteredQuery(matchAllQuery(), typeQuery("type1"))).get(), 2l);
assertHitCount(client().prepareCount().setQuery(filteredQuery(matchAllQuery(), typeQuery("type2"))).get(), 3l);
assertHitCount(client().prepareCount().setTypes("type1").setQuery(matchAllQuery()).get(), 2l);
assertHitCount(client().prepareCount().setTypes("type2").setQuery(matchAllQuery()).get(), 3l);
assertHitCount(client().prepareCount().setTypes("type1", "type2").setQuery(matchAllQuery()).get(), 5l);
}
@Test
public void idsQueryTestsIdIndexed() throws Exception {
idsQueryTests("not_analyzed");
}
@Test
public void idsQueryTestsIdNotIndexed() throws Exception {
idsQueryTests("no");
}
private void idsQueryTests(String index) throws Exception {
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
assertAcked(prepareCreate("test").setSettings(indexSettings)
.addMapping("type1", jsonBuilder().startObject().startObject("type1")
.startObject("_id").field("index", index).endObject()
.endObject().endObject()));
indexRandom(true, client().prepareIndex("test", "type1", "1").setSource("field1", "value1"),
client().prepareIndex("test", "type1", "2").setSource("field1", "value2"),
client().prepareIndex("test", "type1", "3").setSource("field1", "value3"));
CountResponse countResponse = client().prepareCount().setQuery(constantScoreQuery(idsQuery("type1").ids("1", "3"))).get();
assertHitCount(countResponse, 2l);
// no type
countResponse = client().prepareCount().setQuery(constantScoreQuery(idsQuery().ids("1", "3"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(idsQuery("type1").ids("1", "3")).get();
assertHitCount(countResponse, 2l);
// no type
countResponse = client().prepareCount().setQuery(idsQuery().ids("1", "3")).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(idsQuery("type1").ids("7", "10")).get();
assertHitCount(countResponse, 0l);
// repeat..., with terms
countResponse = client().prepareCount().setTypes("type1").setQuery(constantScoreQuery(termsQuery("_id", "1", "3"))).get();
assertHitCount(countResponse, 2l);
}
@Test
public void testLimitFilter() throws Exception {
assertAcked(prepareCreate("test").setSettings(SETTING_NUMBER_OF_SHARDS, 1));
indexRandom(true, client().prepareIndex("test", "type1", "1").setSource("field1", "value1_1"),
client().prepareIndex("test", "type1", "2").setSource("field1", "value1_2"),
client().prepareIndex("test", "type1", "3").setSource("field2", "value2_3"),
client().prepareIndex("test", "type1", "4").setSource("field3", "value3_4"));
CountResponse countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), limitQuery(2))).get();
assertHitCount(countResponse, 4l); // limit is a no-op
}
@Test
public void filterExistsMissingTests() throws Exception {
createIndex("test");
indexRandom(true,
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()),
client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject().startObject("obj1").field("obj1_val", "1").endObject().field("x2", "x_2").field("field1", "value1_2").endObject()),
client().prepareIndex("test", "type1", "3").setSource(jsonBuilder().startObject().startObject("obj2").field("obj2_val", "1").endObject().field("y1", "y_1").field("field2", "value2_3").endObject()),
client().prepareIndex("test", "type1", "4").setSource(jsonBuilder().startObject().startObject("obj2").field("obj2_val", "1").endObject().field("y2", "y_2").field("field3", "value3_4").endObject()));
CountResponse countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), existsQuery("field1"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(constantScoreQuery(existsQuery("field1"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(queryStringQuery("_exists_:field1")).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), existsQuery("field2"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), existsQuery("field3"))).get();
assertHitCount(countResponse, 1l);
// wildcard check
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), existsQuery("x*"))).get();
assertHitCount(countResponse, 2l);
// object check
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), existsQuery("obj1"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), missingQuery("field1"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), missingQuery("field1"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(constantScoreQuery(missingQuery("field1"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(queryStringQuery("_missing_:field1")).get();
assertHitCount(countResponse, 2l);
// wildcard check
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), missingQuery("x*"))).get();
assertHitCount(countResponse, 2l);
// object check
countResponse = client().prepareCount().setQuery(filteredQuery(matchAllQuery(), missingQuery("obj1"))).get();
assertHitCount(countResponse, 2l);
}
@Test
public void passQueryAsJSONStringTest() throws Exception {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("field1", "value1_1", "field2", "value2_1").setRefresh(true).get();
WrapperQueryBuilder wrapper = new WrapperQueryBuilder("{ \"term\" : { \"field1\" : \"value1_1\" } }");
assertHitCount(client().prepareCount().setQuery(wrapper).get(), 1l);
BoolQueryBuilder bool = boolQuery().must(wrapper).must(new TermQueryBuilder("field2", "value2_1"));
assertHitCount(client().prepareCount().setQuery(bool).get(), 1l);
}
@Test
public void testMatchQueryNumeric() throws Exception {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("long", 1l, "double", 1.0d).get();
client().prepareIndex("test", "type1", "2").setSource("long", 2l, "double", 2.0d).get();
client().prepareIndex("test", "type1", "3").setSource("long", 3l, "double", 3.0d).get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(matchQuery("long", "1")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(matchQuery("double", "2")).get();
assertHitCount(countResponse, 1l);
}
@Test
public void testMultiMatchQuery() throws Exception {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("field1", "value1", "field2", "value4", "field3", "value3").get();
client().prepareIndex("test", "type1", "2").setSource("field1", "value2", "field2", "value5", "field3", "value2").get();
client().prepareIndex("test", "type1", "3").setSource("field1", "value3", "field2", "value6", "field3", "value1").get();
refresh();
MultiMatchQueryBuilder builder = QueryBuilders.multiMatchQuery("value1 value2 value4", "field1", "field2");
CountResponse countResponse = client().prepareCount().setQuery(builder).get();
assertHitCount(countResponse, 2l);
refresh();
builder = QueryBuilders.multiMatchQuery("value1", "field1", "field2")
.operator(MatchQueryBuilder.Operator.AND); // Operator only applies on terms inside a field! Fields are always OR-ed together.
countResponse = client().prepareCount().setQuery(builder).get();
assertHitCount(countResponse, 1l);
refresh();
builder = QueryBuilders.multiMatchQuery("value1", "field1", "field3^1.5")
.operator(MatchQueryBuilder.Operator.AND); // Operator only applies on terms inside a field! Fields are always OR-ed together.
countResponse = client().prepareCount().setQuery(builder).get();
assertHitCount(countResponse, 2l);
refresh();
builder = QueryBuilders.multiMatchQuery("value1").field("field1").field("field3", 1.5f)
.operator(MatchQueryBuilder.Operator.AND); // Operator only applies on terms inside a field! Fields are always OR-ed together.
countResponse = client().prepareCount().setQuery(builder).get();
assertHitCount(countResponse, 2l);
// Test lenient
client().prepareIndex("test", "type1", "3").setSource("field1", "value7", "field2", "value8", "field4", 5).get();
refresh();
builder = QueryBuilders.multiMatchQuery("value1", "field1", "field2", "field4");
builder.lenient(true);
countResponse = client().prepareCount().setQuery(builder).get();
assertHitCount(countResponse, 1l);
}
@Test
public void testMatchQueryZeroTermsQuery() {
assertAcked(prepareCreate("test")
.addMapping("type1", "field1", "type=string,analyzer=classic", "field2", "type=string,analyzer=classic"));
client().prepareIndex("test", "type1", "1").setSource("field1", "value1").get();
client().prepareIndex("test", "type1", "2").setSource("field1", "value2").get();
refresh();
BoolQueryBuilder boolQuery = boolQuery()
.must(matchQuery("field1", "a").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.NONE))
.must(matchQuery("field1", "value1").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.NONE));
CountResponse countResponse = client().prepareCount().setQuery(boolQuery).get();
assertHitCount(countResponse, 0l);
boolQuery = boolQuery()
.must(matchQuery("field1", "a").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.ALL))
.must(matchQuery("field1", "value1").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.ALL));
countResponse = client().prepareCount().setQuery(boolQuery).get();
assertHitCount(countResponse, 1l);
boolQuery = boolQuery().must(matchQuery("field1", "a").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.ALL));
countResponse = client().prepareCount().setQuery(boolQuery).get();
assertHitCount(countResponse, 2l);
}
@Test
public void testMultiMatchQueryZeroTermsQuery() {
assertAcked(prepareCreate("test")
.addMapping("type1", "field1", "type=string,analyzer=classic", "field2", "type=string,analyzer=classic"));
client().prepareIndex("test", "type1", "1").setSource("field1", "value1", "field2", "value2").get();
client().prepareIndex("test", "type1", "2").setSource("field1", "value3", "field2", "value4").get();
refresh();
BoolQueryBuilder boolQuery = boolQuery()
.must(multiMatchQuery("a", "field1", "field2").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.NONE))
.must(multiMatchQuery("value1", "field1", "field2").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.NONE)); // Fields are ORed together
CountResponse countResponse = client().prepareCount().setQuery(boolQuery).get();
assertHitCount(countResponse, 0l);
boolQuery = boolQuery()
.must(multiMatchQuery("a", "field1", "field2").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.ALL))
.must(multiMatchQuery("value4", "field1", "field2").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.ALL));
countResponse = client().prepareCount().setQuery(boolQuery).get();
assertHitCount(countResponse, 1l);
boolQuery = boolQuery().must(multiMatchQuery("a", "field1").zeroTermsQuery(MatchQueryBuilder.ZeroTermsQuery.ALL));
countResponse = client().prepareCount().setQuery(boolQuery).get();
assertHitCount(countResponse, 2l);
}
@Test
public void testMultiMatchQueryMinShouldMatch() {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("field1", new String[]{"value1", "value2", "value3"}).get();
client().prepareIndex("test", "type1", "2").setSource("field2", "value1").get();
refresh();
MultiMatchQueryBuilder multiMatchQuery = multiMatchQuery("value1 value2 foo", "field1", "field2");
multiMatchQuery.useDisMax(true);
multiMatchQuery.minimumShouldMatch("70%");
CountResponse countResponse = client().prepareCount().setQuery(multiMatchQuery).get();
assertHitCount(countResponse, 1l);
multiMatchQuery.minimumShouldMatch("30%");
countResponse = client().prepareCount().setQuery(multiMatchQuery).get();
assertHitCount(countResponse, 2l);
multiMatchQuery.useDisMax(false);
multiMatchQuery.minimumShouldMatch("70%");
countResponse = client().prepareCount().setQuery(multiMatchQuery).get();
assertHitCount(countResponse, 1l);
multiMatchQuery.minimumShouldMatch("30%");
countResponse = client().prepareCount().setQuery(multiMatchQuery).get();
assertHitCount(countResponse, 2l);
multiMatchQuery = multiMatchQuery("value1 value2 bar", "field1");
multiMatchQuery.minimumShouldMatch("100%");
countResponse = client().prepareCount().setQuery(multiMatchQuery).get();
assertHitCount(countResponse, 0l);
multiMatchQuery.minimumShouldMatch("70%");
countResponse = client().prepareCount().setQuery(multiMatchQuery).get();
assertHitCount(countResponse, 1l);
}
@Test
public void testFuzzyQueryString() {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("str", "kimchy", "date", "2012-02-01", "num", 12).get();
client().prepareIndex("test", "type1", "2").setSource("str", "shay", "date", "2012-02-05", "num", 20).get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(queryStringQuery("str:kimcy~1")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("num:11~1")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("date:2012-02-02~1d")).get();
assertHitCount(countResponse, 1l);
}
@Test
public void testSpecialRangeSyntaxInQueryString() {
createIndex("test");
client().prepareIndex("test", "type1", "1").setSource("str", "kimchy", "date", "2012-02-01", "num", 12).get();
client().prepareIndex("test", "type1", "2").setSource("str", "shay", "date", "2012-02-05", "num", 20).get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(queryStringQuery("num:>19")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("num:>20")).get();
assertHitCount(countResponse, 0l);
countResponse = client().prepareCount().setQuery(queryStringQuery("num:>=20")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("num:>11")).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(queryStringQuery("num:<20")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(queryStringQuery("num:<=20")).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(queryStringQuery("+num:>11 +num:<20")).get();
assertHitCount(countResponse, 1l);
}
@Test
public void testEmptyTermsFilter() throws Exception {
assertAcked(prepareCreate("test").addMapping("type", "terms", "type=string"));
ensureGreen();
client().prepareIndex("test", "type", "1").setSource("term", "1").get();
client().prepareIndex("test", "type", "2").setSource("term", "2").get();
client().prepareIndex("test", "type", "3").setSource("term", "3").get();
client().prepareIndex("test", "type", "4").setSource("term", "4").get();
refresh();
CountResponse countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsQuery("term", new String[0]))).get();
assertHitCount(countResponse, 0l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), idsQuery())).get();
assertHitCount(countResponse, 0l);
}
@Test
public void testTermsLookupFilter() throws Exception {
assertAcked(prepareCreate("lookup").addMapping("type", "terms", "type=string", "other", "type=string"));
assertAcked(prepareCreate("lookup2").addMapping("type",
jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("arr").startObject("properties").startObject("term").field("type", "string")
.endObject().endObject().endObject().endObject().endObject().endObject()));
assertAcked(prepareCreate("test").addMapping("type", "term", "type=string"));
ensureGreen();
indexRandom(true, client().prepareIndex("lookup", "type", "1").setSource("terms", new String[]{"1", "3"}),
client().prepareIndex("lookup", "type", "2").setSource("terms", new String[]{"2"}),
client().prepareIndex("lookup", "type", "3").setSource("terms", new String[]{"2", "4"}),
client().prepareIndex("lookup", "type", "4").setSource("other", "value"),
client().prepareIndex("lookup2", "type", "1").setSource(XContentFactory.jsonBuilder().startObject()
.startArray("arr")
.startObject().field("term", "1").endObject()
.startObject().field("term", "3").endObject()
.endArray()
.endObject()),
client().prepareIndex("lookup2", "type", "2").setSource(XContentFactory.jsonBuilder().startObject()
.startArray("arr")
.startObject().field("term", "2").endObject()
.endArray()
.endObject()),
client().prepareIndex("lookup2", "type", "3").setSource(XContentFactory.jsonBuilder().startObject()
.startArray("arr")
.startObject().field("term", "2").endObject()
.startObject().field("term", "4").endObject()
.endArray()
.endObject()),
client().prepareIndex("test", "type", "1").setSource("term", "1"),
client().prepareIndex("test", "type", "2").setSource("term", "2"),
client().prepareIndex("test", "type", "3").setSource("term", "3"),
client().prepareIndex("test", "type", "4").setSource("term", "4"));
CountResponse countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms"))).get();
assertHitCount(countResponse, 2l);
// same as above, just on the _id...
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("_id").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms"))).get();
assertHitCount(countResponse, 2l);
// another search with same parameters...
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup").lookupType("type").lookupId("2").lookupPath("terms"))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup").lookupType("type").lookupId("3").lookupPath("terms"))
).get();
assertNoFailures(countResponse);
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup").lookupType("type").lookupId("4").lookupPath("terms"))).get();
assertHitCount(countResponse, 0l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup2").lookupType("type").lookupId("1").lookupPath("arr.term"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup2").lookupType("type").lookupId("2").lookupPath("arr.term"))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("term").lookupIndex("lookup2").lookupType("type").lookupId("3").lookupPath("arr.term"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount("test")
.setQuery(filteredQuery(matchAllQuery(), termsLookupQuery("not_exists").lookupIndex("lookup2").lookupType("type").lookupId("3").lookupPath("arr.term"))).get();
assertHitCount(countResponse, 0l);
}
@Test
public void testBasicFilterById() throws Exception {
createIndex("test");
ensureGreen();
client().prepareIndex("test", "type1", "1").setSource("field1", "value1").get();
client().prepareIndex("test", "type2", "2").setSource("field1", "value2").get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(QueryBuilders.constantScoreQuery(QueryBuilders.idsQuery("type1", "type2").ids("1", "2"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(QueryBuilders.constantScoreQuery(QueryBuilders.idsQuery().ids("1", "2"))).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(QueryBuilders.constantScoreQuery(QueryBuilders.idsQuery("type1").ids("1", "2"))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.constantScoreQuery(QueryBuilders.idsQuery().ids("1"))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.constantScoreQuery(QueryBuilders.idsQuery(null).ids("1"))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.constantScoreQuery(QueryBuilders.idsQuery("type1", "type2", "type3").ids("1", "2", "3", "4"))).get();
assertHitCount(countResponse, 2l);
}
@Test
public void testBasicQueryById() throws Exception {
createIndex("test");
ensureGreen();
client().prepareIndex("test", "type1", "1").setSource("field1", "value1").get();
client().prepareIndex("test", "type2", "2").setSource("field1", "value2").get();
refresh();
CountResponse countResponse = client().prepareCount().setQuery(QueryBuilders.idsQuery("type1", "type2").ids("1", "2")).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(QueryBuilders.idsQuery().ids("1")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.idsQuery().ids("1", "2")).get();
assertHitCount(countResponse, 2l);
countResponse = client().prepareCount().setQuery(QueryBuilders.idsQuery("type1").ids("1", "2")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.idsQuery().ids("1")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.idsQuery(null).ids("1")).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount().setQuery(QueryBuilders.idsQuery("type1", "type2", "type3").ids("1", "2", "3", "4")).get();
assertHitCount(countResponse, 2l);
}
@Test
public void testNumericTermsAndRanges() throws Exception {
assertAcked(prepareCreate("test")
.addMapping("type1",
"num_byte", "type=byte", "num_short", "type=short",
"num_integer", "type=integer", "num_long", "type=long",
"num_float", "type=float", "num_double", "type=double"));
ensureGreen();
client().prepareIndex("test", "type1", "1").setSource("num_byte", 1, "num_short", 1, "num_integer", 1,
"num_long", 1, "num_float", 1, "num_double", 1).get();
client().prepareIndex("test", "type1", "2").setSource("num_byte", 2, "num_short", 2, "num_integer", 2,
"num_long", 2, "num_float", 2, "num_double", 2).get();
client().prepareIndex("test", "type1", "17").setSource("num_byte", 17, "num_short", 17, "num_integer", 17,
"num_long", 17, "num_float", 17, "num_double", 17).get();
refresh();
CountResponse countResponse;
logger.info("--> term query on 1");
countResponse = client().prepareCount("test").setQuery(termQuery("num_byte", 1)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termQuery("num_short", 1)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termQuery("num_integer", 1)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termQuery("num_long", 1)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termQuery("num_float", 1)).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termQuery("num_double", 1)).get();
assertHitCount(countResponse, 1l);
logger.info("--> terms query on 1");
countResponse = client().prepareCount("test").setQuery(termsQuery("num_byte", new int[]{1})).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termsQuery("num_short", new int[]{1})).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termsQuery("num_integer", new int[]{1})).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termsQuery("num_long", new int[]{1})).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termsQuery("num_float", new double[]{1})).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(termsQuery("num_double", new double[]{1})).get();
assertHitCount(countResponse, 1l);
logger.info("--> term filter on 1");
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termQuery("num_byte", 1))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termQuery("num_short", 1))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termQuery("num_integer", 1))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termQuery("num_long", 1))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termQuery("num_float", 1))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termQuery("num_double", 1))).get();
assertHitCount(countResponse, 1l);
logger.info("--> terms filter on 1");
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termsQuery("num_byte", new int[]{1}))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termsQuery("num_short", new int[]{1}))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termsQuery("num_integer", new int[]{1}))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termsQuery("num_long", new int[]{1}))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termsQuery("num_float", new int[]{1}))).get();
assertHitCount(countResponse, 1l);
countResponse = client().prepareCount("test").setQuery(filteredQuery(matchAllQuery(), termsQuery("num_double", new int[]{1}))).get();
assertHitCount(countResponse, 1l);
}
@Test // see #2994
public void testSimpleSpan() throws IOException {
createIndex("test");
ensureGreen();
client().prepareIndex("test", "test", "1").setSource("description", "foo other anything bar").get();
client().prepareIndex("test", "test", "2").setSource("description", "foo other anything").get();
client().prepareIndex("test", "test", "3").setSource("description", "foo other").get();
client().prepareIndex("test", "test", "4").setSource("description", "foo").get();
refresh();
CountResponse response = client().prepareCount("test")
.setQuery(QueryBuilders.spanOrQuery().clause(QueryBuilders.spanTermQuery("description", "bar"))).get();
assertHitCount(response, 1l);
response = client().prepareCount("test").setQuery(
QueryBuilders.spanNearQuery()
.clause(QueryBuilders.spanTermQuery("description", "foo"))
.clause(QueryBuilders.spanTermQuery("description", "other"))
.slop(3)).get();
assertHitCount(response, 3l);
}
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.document;
import com.google.common.base.Charsets;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse;
import org.elasticsearch.action.admin.indices.flush.FlushResponse;
@ -30,6 +29,7 @@ import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
@ -40,7 +40,8 @@ import java.io.IOException;
import static org.elasticsearch.client.Requests.*;
import static org.elasticsearch.index.query.QueryBuilders.termQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue;
/**
*
@ -163,11 +164,11 @@ public class DocumentActionsTests extends ElasticsearchIntegrationTest {
assertThat(countResponse.getFailedShards(), equalTo(0));
// test failed (simply query that can't be parsed)
countResponse = client().count(countRequest("test").source("{ term : { _type : \"type1 } }".getBytes(Charsets.UTF_8))).actionGet();
assertThat(countResponse.getCount(), equalTo(0l));
assertThat(countResponse.getSuccessfulShards(), equalTo(0));
assertThat(countResponse.getFailedShards(), equalTo(numShards.numPrimaries));
try {
client().count(countRequest("test").source("{ term : { _type : \"type1 } }")).actionGet();
} catch(SearchPhaseExecutionException e) {
assertThat(e.shardFailures().length, equalTo(numShards.numPrimaries));
}
// count with no query is a match all one
countResponse = client().prepareCount("test").execute().actionGet();