From c222ce28fc2245216fb21f4f7250e1a67872d79a Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 16 Jul 2013 16:00:28 +0200 Subject: [PATCH] Redesigned the percolator engine to execute in a distribute manner. With this design the percolate queries will be stored in a special `_percolator` type with its own mapping in the same index where the actual data is or in a different index (dedicated percolation index, which might require different sharding behavior compared to the index that holds actual data and being search on). This approach allows percolate requests to scale to the number of primary shards an index has been configured with and effectively distributes the percolate execution. This commit doesn't add new percolate features other than scaling. The response remains similar, with exception that a header similar to the search api has been added to the percolate response. Closes #3173 --- ...oryIndex.java => ExtendedMemoryIndex.java} | 14 +- .../delete/TransportDeleteIndexAction.java | 22 +- .../action/bulk/BulkRequest.java | 14 +- .../action/bulk/TransportShardBulkAction.java | 45 +- .../action/index/IndexRequest.java | 17 - .../action/index/IndexRequestBuilder.java | 10 - .../action/index/IndexResponse.java | 48 -- .../action/index/TransportIndexAction.java | 18 - .../action/percolate/PercolateRequest.java | 131 ++-- .../percolate/PercolateRequestBuilder.java | 63 +- .../action/percolate/PercolateResponse.java | 59 +- .../percolate/PercolateShardRequest.java | 48 ++ .../percolate/PercolateShardResponse.java | 43 ++ .../percolate/TransportPercolateAction.java | 128 +++- .../action/update/TransportUpdateAction.java | 8 +- .../action/update/UpdateHelper.java | 2 - .../action/update/UpdateRequest.java | 18 - .../action/update/UpdateRequestBuilder.java | 10 - .../action/update/UpdateResponse.java | 48 -- .../metadata/MetaDataCreateIndexService.java | 48 +- .../metadata/MetaDataMappingService.java | 3 +- .../common/io/stream/StreamInput.java | 13 + .../common/io/stream/StreamOutput.java | 7 + .../DocumentTypeListener.java} | 30 +- .../index/mapper/MapperService.java | 104 +++- .../index/percolator/PercolateException.java | 26 + .../index/percolator/PercolatorException.java | 2 +- .../index/percolator/PercolatorExecutor.java | 575 ------------------ .../index/percolator/PercolatorModule.java | 1 - .../percolator/PercolatorQueriesRegistry.java | 302 +++++++++ .../index/percolator/PercolatorService.java | 405 +++++------- .../percolator/PercolatorShardModule.java | 32 + .../percolator/QueriesLoaderCollector.java | 70 +++ .../index/percolator/QueryCollector.java | 87 +++ .../index/service/IndexService.java | 3 - .../index/service/InternalIndexService.java | 26 +- .../index/shard/service/IndexShard.java | 3 + .../shard/service/InternalIndexShard.java | 12 +- .../indices/InternalIndicesService.java | 6 +- .../node/internal/InternalNode.java | 5 + .../rest/action/bulk/RestBulkAction.java | 13 +- .../rest/action/index/RestIndexAction.java | 8 - .../action/percolate/RestPercolateAction.java | 43 +- .../rest/action/update/RestUpdateAction.java | 8 - .../elasticsearch/threadpool/ThreadPool.java | 2 +- .../EmbeddedPercolatorBenchmarkTest.java | 137 ----- ...rk.java => PercolatorStressBenchmark.java} | 99 ++- .../test/integration/TestCluster.java | 44 +- .../percolator/ConcurrentPercolatorTests.java | 378 ++++++++++++ .../percolator/RecoveryPercolatorTests.java | 86 ++- .../percolator/SimplePercolatorTests.java | 515 ++++++++-------- .../percolator/TTLPercolatorTests.java | 99 +++ .../test/integration/update/UpdateTests.java | 14 - .../index/cache/id/SimpleIdCacheTests.java | 6 - .../percolator/PercolatorExecutorTests.java | 252 -------- 55 files changed, 2157 insertions(+), 2053 deletions(-) rename src/main/java/org/apache/lucene/index/memory/{ReusableMemoryIndex.java => ExtendedMemoryIndex.java} (76%) create mode 100644 src/main/java/org/elasticsearch/action/percolate/PercolateShardRequest.java create mode 100644 src/main/java/org/elasticsearch/action/percolate/PercolateShardResponse.java rename src/main/java/org/elasticsearch/index/{percolator/PercolateIndexUnavailable.java => mapper/DocumentTypeListener.java} (52%) create mode 100644 src/main/java/org/elasticsearch/index/percolator/PercolateException.java delete mode 100644 src/main/java/org/elasticsearch/index/percolator/PercolatorExecutor.java create mode 100644 src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java create mode 100644 src/main/java/org/elasticsearch/index/percolator/PercolatorShardModule.java create mode 100644 src/main/java/org/elasticsearch/index/percolator/QueriesLoaderCollector.java create mode 100644 src/main/java/org/elasticsearch/index/percolator/QueryCollector.java delete mode 100644 src/test/java/org/elasticsearch/benchmark/percolator/EmbeddedPercolatorBenchmarkTest.java rename src/test/java/org/elasticsearch/benchmark/percolator/{SinglePercolatorStressBenchmark.java => PercolatorStressBenchmark.java} (51%) create mode 100644 src/test/java/org/elasticsearch/test/integration/percolator/ConcurrentPercolatorTests.java create mode 100644 src/test/java/org/elasticsearch/test/integration/percolator/TTLPercolatorTests.java delete mode 100644 src/test/java/org/elasticsearch/test/unit/index/percolator/PercolatorExecutorTests.java diff --git a/src/main/java/org/apache/lucene/index/memory/ReusableMemoryIndex.java b/src/main/java/org/apache/lucene/index/memory/ExtendedMemoryIndex.java similarity index 76% rename from src/main/java/org/apache/lucene/index/memory/ReusableMemoryIndex.java rename to src/main/java/org/apache/lucene/index/memory/ExtendedMemoryIndex.java index 5fa1c8eed3d8a..bbd66fe8e81b0 100644 --- a/src/main/java/org/apache/lucene/index/memory/ReusableMemoryIndex.java +++ b/src/main/java/org/apache/lucene/index/memory/ExtendedMemoryIndex.java @@ -20,18 +20,12 @@ */ /** - * This class overwrites {@link MemoryIndex} to make the reuse constructor - * visible. + * This class overwrites {@link MemoryIndex} to make the reuse constructor visible. */ -public final class ReusableMemoryIndex extends MemoryIndex { - private final long maxReuseBytes; - public ReusableMemoryIndex(boolean storeOffsets, long maxReusedBytes) { +public final class ExtendedMemoryIndex extends MemoryIndex { + + public ExtendedMemoryIndex(boolean storeOffsets, long maxReusedBytes) { super(storeOffsets, maxReusedBytes); - this.maxReuseBytes = maxReusedBytes; - } - - public long getMaxReuseBytes() { - return maxReuseBytes; } } diff --git a/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java b/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java index f01ef16ba2493..4a13ae153c936 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/delete/TransportDeleteIndexAction.java @@ -22,19 +22,15 @@ import org.elasticsearch.ElasticSearchException; import org.elasticsearch.ElasticSearchIllegalArgumentException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.indices.mapping.delete.DeleteMappingRequest; -import org.elasticsearch.action.admin.indices.mapping.delete.DeleteMappingResponse; import org.elasticsearch.action.admin.indices.mapping.delete.TransportDeleteMappingAction; import org.elasticsearch.action.support.master.TransportMasterNodeOperationAction; 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.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaDataDeleteIndexService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -116,23 +112,7 @@ protected DeleteIndexResponse masterOperation(DeleteIndexRequest request, final @Override public void onResponse(MetaDataDeleteIndexService.Response response) { responseRef.set(new DeleteIndexResponse(response.acknowledged())); - // YACK, but here we go: If this index is also percolated, make sure to delete all percolated queries from the _percolator index - IndexMetaData percolatorMetaData = state.metaData().index(PercolatorService.INDEX_NAME); - if (percolatorMetaData != null && percolatorMetaData.mappings().containsKey(index)) { - deleteMappingAction.execute(new DeleteMappingRequest(PercolatorService.INDEX_NAME).type(index), new ActionListener() { - @Override - public void onResponse(DeleteMappingResponse deleteMappingResponse) { - latch.countDown(); - } - - @Override - public void onFailure(Throwable e) { - latch.countDown(); - } - }); - } else { - latch.countDown(); - } + latch.countDown(); } @Override diff --git a/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index a988a57cbd2c4..42be0e4bed7fb 100644 --- a/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -311,8 +311,6 @@ public BulkRequest add(BytesReference data, boolean contentUnsafe, @Nullable Str version = parser.longValue(); } else if ("_version_type".equals(currentFieldName) || "_versionType".equals(currentFieldName) || "version_type".equals(currentFieldName) || "versionType".equals(currentFieldName)) { versionType = VersionType.fromString(parser.text()); - } else if ("percolate".equals(currentFieldName) || "_percolate".equals(currentFieldName)) { - percolate = parser.textOrNull(); } else if ("_retry_on_conflict".equals(currentFieldName) || "_retryOnConflict".equals(currentFieldName)) { retryOnConflict = parser.intValue(); } @@ -332,24 +330,20 @@ public BulkRequest add(BytesReference data, boolean contentUnsafe, @Nullable Str if ("index".equals(action)) { if (opType == null) { internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType) - .source(data.slice(from, nextMarker - from), contentUnsafe) - .percolate(percolate), payload); + .source(data.slice(from, nextMarker - from), contentUnsafe), payload); } else { internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType) .create("create".equals(opType)) - .source(data.slice(from, nextMarker - from), contentUnsafe) - .percolate(percolate), payload); + .source(data.slice(from, nextMarker - from), contentUnsafe), payload); } } else if ("create".equals(action)) { internalAdd(new IndexRequest(index, type, id).routing(routing).parent(parent).timestamp(timestamp).ttl(ttl).version(version).versionType(versionType) .create(true) - .source(data.slice(from, nextMarker - from), contentUnsafe) - .percolate(percolate), payload); + .source(data.slice(from, nextMarker - from), contentUnsafe), payload); } else if ("update".equals(action)) { internalAdd(new UpdateRequest(index, type, id).routing(routing).parent(parent).retryOnConflict(retryOnConflict) .version(version).versionType(versionType) - .source(data.slice(from, nextMarker - from)) - .percolate(percolate), payload); + .source(data.slice(from, nextMarker - from)), payload); } // move pointers from = nextMarker + 1; diff --git a/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index 201cd90fd406b..1347aba6a7980 100644 --- a/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -42,7 +42,6 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.routing.ShardIterator; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; @@ -55,8 +54,6 @@ import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.percolator.PercolatorExecutor; -import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.indices.IndicesService; @@ -232,9 +229,7 @@ protected PrimaryResponse shardOperationOnP BytesReference indexSourceAsBytes = indexRequest.source(); // add the response IndexResponse indexResponse = result.response(); - UpdateResponse updateResponse = new UpdateResponse(indexResponse.getIndex(), indexResponse.getType(), - indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated()); - updateResponse.setMatches(indexResponse.getMatches()); + UpdateResponse updateResponse = new UpdateResponse(indexResponse.getIndex(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated()); if (updateRequest.fields() != null && updateRequest.fields().length > 0) { Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, indexResponse.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), indexSourceAsBytes)); @@ -402,11 +397,6 @@ private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest i mappingsToUpdate = Tuple.tuple(indexRequest.index(), indexRequest.type()); } - // if we are going to percolate, then we need to keep this op for the postPrimary operation - if (!Strings.hasLength(indexRequest.percolate())) { - op = null; - } - IndexResponse indexResponse = new IndexResponse(indexRequest.index(), indexRequest.type(), indexRequest.id(), version, created); return new WriteResult(indexResponse, preVersion, mappingsToUpdate, op); } @@ -511,39 +501,6 @@ private UpdateResult shardUpdateOperation(ClusterState clusterState, BulkShardRe } } - @Override - protected void postPrimaryOperation(BulkShardRequest request, PrimaryResponse response) { - IndexService indexService = indicesService.indexServiceSafe(request.index()); - Engine.IndexingOperation[] ops = (Engine.IndexingOperation[]) response.payload(); - if (ops == null) { - return; - } - for (int i = 0; i < ops.length; i++) { - BulkItemRequest itemRequest = request.items()[i]; - BulkItemResponse itemResponse = response.response().getResponses()[i]; - if (itemResponse.isFailed()) { - // failure, continue - continue; - } - Engine.IndexingOperation op = ops[i]; - if (op == null) { - continue; // failed / no matches requested - } - if (itemRequest.request() instanceof IndexRequest) { - IndexRequest indexRequest = (IndexRequest) itemRequest.request(); - if (!Strings.hasLength(indexRequest.percolate())) { - continue; - } - try { - PercolatorExecutor.Response percolate = indexService.percolateService().percolate(new PercolatorExecutor.DocAndSourceQueryRequest(op.parsedDoc(), indexRequest.percolate())); - ((IndexResponse) itemResponse.getResponse()).setMatches(percolate.matches()); - } catch (Exception e) { - logger.warn("failed to percolate [{}]", e, itemRequest.request()); - } - } - } - } - @Override protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); diff --git a/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/src/main/java/org/elasticsearch/action/index/IndexRequest.java index f384e1e835968..c4a8dcb9cfe84 100644 --- a/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -133,7 +133,6 @@ public static OpType fromId(byte id) { private boolean refresh = false; private long version = Versions.MATCH_ANY; private VersionType versionType = VersionType.INTERNAL; - private String percolate; private XContentType contentType = Requests.INDEX_CONTENT_TYPE; @@ -549,20 +548,6 @@ public VersionType versionType() { return this.versionType; } - /** - * Causes the index request document to be percolated. The parameter is the percolate query - * to use to reduce the percolated queries that are going to run against this doc. Can be - * set to * to indicate that all percolate queries should be run. - */ - public IndexRequest percolate(String percolate) { - this.percolate = percolate; - return this; - } - - public String percolate() { - return this.percolate; - } - public void process(MetaData metaData, String aliasOrIndex, @Nullable MappingMetaData mappingMd, boolean allowIdGeneration) throws ElasticSearchException { // resolve the routing if needed routing(metaData.resolveIndexRouting(routing, aliasOrIndex)); @@ -635,7 +620,6 @@ public void readFrom(StreamInput in) throws IOException { opType = OpType.fromId(in.readByte()); refresh = in.readBoolean(); version = in.readLong(); - percolate = in.readOptionalString(); versionType = VersionType.fromValue(in.readByte()); } @@ -652,7 +636,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeByte(opType.id()); out.writeBoolean(refresh); out.writeLong(version); - out.writeOptionalString(percolate); out.writeByte(versionType.getValue()); } diff --git a/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java b/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java index 6445b3fb833dd..2a847b657fb7d 100644 --- a/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java @@ -294,16 +294,6 @@ public IndexRequestBuilder setVersionType(VersionType versionType) { return this; } - /** - * Causes the index request document to be percolated. The parameter is the percolate query - * to use to reduce the percolated queries that are going to run against this doc. Can be - * set to * to indicate that all percolate queries should be run. - */ - public IndexRequestBuilder setPercolate(String percolate) { - request.percolate(percolate); - return this; - } - /** * Sets the timestamp either as millis since the epoch, or, in the configured date format. */ diff --git a/src/main/java/org/elasticsearch/action/index/IndexResponse.java b/src/main/java/org/elasticsearch/action/index/IndexResponse.java index 6b2a39fd129f8..b348d02257b91 100644 --- a/src/main/java/org/elasticsearch/action/index/IndexResponse.java +++ b/src/main/java/org/elasticsearch/action/index/IndexResponse.java @@ -19,14 +19,11 @@ package org.elasticsearch.action.index; -import com.google.common.collect.ImmutableList; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; /** * A response of an index operation, @@ -41,7 +38,6 @@ public class IndexResponse extends ActionResponse { private String type; private long version; private boolean created; - private List matches; public IndexResponse() { @@ -90,20 +86,6 @@ public boolean isCreated() { return this.created; } - /** - * Returns the percolate queries matches. null if no percolation was requested. - */ - public List getMatches() { - return this.matches; - } - - /** - * Internal. - */ - public void setMatches(List matches) { - this.matches = matches; - } - @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -112,27 +94,6 @@ public void readFrom(StreamInput in) throws IOException { type = in.readString(); version = in.readLong(); created = in.readBoolean(); - if (in.readBoolean()) { - int size = in.readVInt(); - if (size == 0) { - matches = ImmutableList.of(); - } else if (size == 1) { - matches = ImmutableList.of(in.readString()); - } else if (size == 2) { - matches = ImmutableList.of(in.readString(), in.readString()); - } else if (size == 3) { - matches = ImmutableList.of(in.readString(), in.readString(), in.readString()); - } else if (size == 4) { - matches = ImmutableList.of(in.readString(), in.readString(), in.readString(), in.readString()); - } else if (size == 5) { - matches = ImmutableList.of(in.readString(), in.readString(), in.readString(), in.readString(), in.readString()); - } else { - matches = new ArrayList(); - for (int i = 0; i < size; i++) { - matches.add(in.readString()); - } - } - } } @Override @@ -143,14 +104,5 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(type); out.writeLong(version); out.writeBoolean(created); - if (matches == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - out.writeVInt(matches.size()); - for (String match : matches) { - out.writeString(match); - } - } } } diff --git a/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 2eeab85b8e91e..a1910b872c57b 100644 --- a/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -36,15 +36,12 @@ import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.routing.ShardIterator; -import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SourceToParse; -import org.elasticsearch.index.percolator.PercolatorExecutor; -import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.shard.service.IndexShard; import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndicesService; @@ -236,21 +233,6 @@ protected PrimaryResponse shardOperationOnPrimary(C return new PrimaryResponse(shardRequest.request, response, op); } - @Override - protected void postPrimaryOperation(IndexRequest request, PrimaryResponse response) { - Engine.IndexingOperation op = (Engine.IndexingOperation) response.payload(); - if (!Strings.hasLength(request.percolate())) { - return; - } - IndexService indexService = indicesService.indexServiceSafe(request.index()); - try { - PercolatorExecutor.Response percolate = indexService.percolateService().percolate(new PercolatorExecutor.DocAndSourceQueryRequest(op.parsedDoc(), request.percolate())); - response.response().setMatches(percolate.matches()); - } catch (Exception e) { - logger.warn("failed to percolate [{}]", e, request); - } - } - @Override protected void shardOperationOnReplica(ReplicaOperationRequest shardRequest) { IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.request.index()).shardSafe(shardRequest.shardId); diff --git a/src/main/java/org/elasticsearch/action/percolate/PercolateRequest.java b/src/main/java/org/elasticsearch/action/percolate/PercolateRequest.java index 45ae1d8594b5f..77eb22a33d5e2 100644 --- a/src/main/java/org/elasticsearch/action/percolate/PercolateRequest.java +++ b/src/main/java/org/elasticsearch/action/percolate/PercolateRequest.java @@ -21,7 +21,7 @@ import org.elasticsearch.ElasticSearchGenerationException; import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.support.single.custom.SingleCustomOperationRequest; +import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest; import org.elasticsearch.common.Required; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -39,45 +39,51 @@ /** * */ -public class PercolateRequest extends SingleCustomOperationRequest { +public class PercolateRequest extends BroadcastOperationRequest { - private String index; - private String type; + private String documentType; + private String routing; + private String preference; - private BytesReference source; - private boolean sourceUnsafe; + private BytesReference documentSource; + private boolean documentUnsafe; - public PercolateRequest() { + // Used internally in order to compute tookInMillis, TransportBroadcastOperationAction itself doesn't allow + // to hold it temporarily in an easy way + long startTime; + PercolateRequest() { } - /** - * Constructs a new percolate request. - * - * @param index The index name - * @param type The document type - */ - public PercolateRequest(String index, String type) { - this.index = index; - this.type = type; + public PercolateRequest(String index, String documentType) { + super(new String[]{index}); + this.documentType = documentType; } - public PercolateRequest index(String index) { - this.index = index; - return this; + public String documentType() { + return documentType; + } + + public void documentType(String type) { + this.documentType = type; } - public PercolateRequest type(String type) { - this.type = type; + public String routing() { + return routing; + } + + public PercolateRequest routing(String routing) { + this.routing = routing; return this; } - public String index() { - return this.index; + public String preference() { + return preference; } - public String type() { - return this.type; + public PercolateRequest preference(String preference) { + this.preference = preference; + return this; } /** @@ -85,78 +91,78 @@ public String type() { */ @Override public void beforeLocalFork() { - if (sourceUnsafe) { - source = source.copyBytesArray(); - sourceUnsafe = false; + if (documentUnsafe) { + documentSource = documentSource.copyBytesArray(); + documentUnsafe = false; } } - public BytesReference source() { - return source; + public BytesReference documentSource() { + return documentSource; } @Required - public PercolateRequest source(Map source) throws ElasticSearchGenerationException { - return source(source, XContentType.SMILE); + public PercolateRequest documentSource(Map document) throws ElasticSearchGenerationException { + return documentSource(document, XContentType.SMILE); } @Required - public PercolateRequest source(Map source, XContentType contentType) throws ElasticSearchGenerationException { + public PercolateRequest documentSource(Map document, XContentType contentType) throws ElasticSearchGenerationException { try { XContentBuilder builder = XContentFactory.contentBuilder(contentType); - builder.map(source); - return source(builder); + builder.map(document); + return documentSource(builder); } catch (IOException e) { - throw new ElasticSearchGenerationException("Failed to generate [" + source + "]", e); + throw new ElasticSearchGenerationException("Failed to generate [" + document + "]", e); } } @Required - public PercolateRequest source(String source) { - this.source = new BytesArray(source); - this.sourceUnsafe = false; + public PercolateRequest documentSource(String document) { + this.documentSource = new BytesArray(document); + this.documentUnsafe = false; return this; } @Required - public PercolateRequest source(XContentBuilder sourceBuilder) { - source = sourceBuilder.bytes(); - sourceUnsafe = false; + public PercolateRequest documentSource(XContentBuilder documentBuilder) { + documentSource = documentBuilder.bytes(); + documentUnsafe = false; return this; } - public PercolateRequest source(byte[] source) { - return source(source, 0, source.length); + public PercolateRequest documentSource(byte[] document) { + return documentSource(document, 0, document.length); } @Required - public PercolateRequest source(byte[] source, int offset, int length) { - return source(source, offset, length, false); + public PercolateRequest documentSource(byte[] source, int offset, int length) { + return documentSource(source, offset, length, false); } @Required - public PercolateRequest source(byte[] source, int offset, int length, boolean unsafe) { - return source(new BytesArray(source, offset, length), unsafe); + public PercolateRequest documentSource(byte[] source, int offset, int length, boolean unsafe) { + return documentSource(new BytesArray(source, offset, length), unsafe); } @Required - public PercolateRequest source(BytesReference source, boolean unsafe) { - this.source = source; - this.sourceUnsafe = unsafe; + public PercolateRequest documentSource(BytesReference source, boolean unsafe) { + this.documentSource = source; + this.documentUnsafe = unsafe; return this; } @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = super.validate(); - if (index == null) { + if (indices == null || indices.length == 0) { validationException = addValidationError("index is missing", validationException); } - if (type == null) { + if (documentType == null) { validationException = addValidationError("type is missing", validationException); } - if (source == null) { - validationException = addValidationError("source is missing", validationException); + if (documentSource == null) { + validationException = addValidationError("documentSource is missing", validationException); } return validationException; } @@ -164,18 +170,17 @@ public ActionRequestValidationException validate() { @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - index = in.readString(); - type = in.readString(); - - sourceUnsafe = false; - source = in.readBytesReference(); + startTime = in.readVLong(); + documentType = in.readString(); + documentUnsafe = false; + documentSource = in.readBytesReference(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(index); - out.writeString(type); - out.writeBytesReference(source); + out.writeVLong(startTime); + out.writeString(documentType); + out.writeBytesReference(documentSource); } } diff --git a/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java b/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java index 16672cf90bb68..25ee793c90eb1 100644 --- a/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/percolate/PercolateRequestBuilder.java @@ -20,9 +20,10 @@ package org.elasticsearch.action.percolate; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.single.custom.SingleCustomOperationRequestBuilder; +import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder; import org.elasticsearch.client.Client; import org.elasticsearch.client.internal.InternalClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; @@ -32,29 +33,55 @@ /** * */ -public class PercolateRequestBuilder extends SingleCustomOperationRequestBuilder { - - public PercolateRequestBuilder(Client client) { - super((InternalClient) client, new PercolateRequest()); - } +public class PercolateRequestBuilder extends BroadcastOperationRequestBuilder { public PercolateRequestBuilder(Client client, String index, String type) { super((InternalClient) client, new PercolateRequest(index, type)); } + PercolateRequestBuilder(Client client) { + super((InternalClient) client, new PercolateRequest()); + } + /** * Sets the index to percolate the document against. */ public PercolateRequestBuilder setIndex(String index) { - request.index(index); + request.indices(index); return this; } /** * Sets the type of the document to percolate. */ - public PercolateRequestBuilder setType(String type) { - request.type(type); + public PercolateRequestBuilder setDocumentType(String type) { + request.documentType(type); + return this; + } + + /** + * A comma separated list of routing values to control the shards the search will be executed on. + */ + public PercolateRequestBuilder setRouting(String routing) { + request.routing(routing); + return this; + } + + /** + * List of routing values to control the shards the search will be executed on. + */ + public PercolateRequestBuilder setRouting(String... routings) { + request.routing(Strings.arrayToCommaDelimitedString(routings)); + return this; + } + + /** + * Sets the preference to execute the search. Defaults to randomize across shards. Can be set to + * _local to prefer local shards, _primary to execute only on primary shards, or + * a custom value, which guarantees that the same order will be used across different requests. + */ + public PercolateRequestBuilder setPreference(String preference) { + request.preference(preference); return this; } @@ -64,7 +91,7 @@ public PercolateRequestBuilder setType(String type) { * @param source The map to index */ public PercolateRequestBuilder setSource(Map source) { - request.source(source); + request.documentSource(source); return this; } @@ -74,7 +101,7 @@ public PercolateRequestBuilder setSource(Map source) { * @param source The map to index */ public PercolateRequestBuilder setSource(Map source, XContentType contentType) { - request.source(source, contentType); + request.documentSource(source, contentType); return this; } @@ -85,7 +112,7 @@ public PercolateRequestBuilder setSource(Map source, XContentTyp * or using the {@link #setSource(byte[])}. */ public PercolateRequestBuilder setSource(String source) { - request.source(source); + request.documentSource(source); return this; } @@ -93,7 +120,7 @@ public PercolateRequestBuilder setSource(String source) { * Sets the content source to index. */ public PercolateRequestBuilder setSource(XContentBuilder sourceBuilder) { - request.source(sourceBuilder); + request.documentSource(sourceBuilder); return this; } @@ -101,7 +128,7 @@ public PercolateRequestBuilder setSource(XContentBuilder sourceBuilder) { * Sets the document to index in bytes form. */ public PercolateRequestBuilder setSource(BytesReference source) { - request.source(source, false); + request.documentSource(source, false); return this; } @@ -109,7 +136,7 @@ public PercolateRequestBuilder setSource(BytesReference source) { * Sets the document to index in bytes form. */ public PercolateRequestBuilder setSource(BytesReference source, boolean unsafe) { - request.source(source, unsafe); + request.documentSource(source, unsafe); return this; } @@ -117,7 +144,7 @@ public PercolateRequestBuilder setSource(BytesReference source, boolean unsafe) * Sets the document to index in bytes form. */ public PercolateRequestBuilder setSource(byte[] source) { - request.source(source); + request.documentSource(source); return this; } @@ -130,7 +157,7 @@ public PercolateRequestBuilder setSource(byte[] source) { * @param length The length of the data */ public PercolateRequestBuilder setSource(byte[] source, int offset, int length) { - request.source(source, offset, length); + request.documentSource(source, offset, length); return this; } @@ -143,7 +170,7 @@ public PercolateRequestBuilder setSource(byte[] source, int offset, int length) * @param unsafe Is the byte array safe to be used form a different thread */ public PercolateRequestBuilder setSource(byte[] source, int offset, int length, boolean unsafe) { - request.source(source, offset, length, unsafe); + request.documentSource(source, offset, length, unsafe); return this; } diff --git a/src/main/java/org/elasticsearch/action/percolate/PercolateResponse.java b/src/main/java/org/elasticsearch/action/percolate/PercolateResponse.java index 76a991d94f882..b45a9c5ef1e87 100644 --- a/src/main/java/org/elasticsearch/action/percolate/PercolateResponse.java +++ b/src/main/java/org/elasticsearch/action/percolate/PercolateResponse.java @@ -19,55 +19,78 @@ package org.elasticsearch.action.percolate; -import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.common.unit.TimeValue; import java.io.IOException; -import java.util.ArrayList; +import java.util.Arrays; import java.util.Iterator; import java.util.List; /** * */ -public class PercolateResponse extends ActionResponse implements Iterable { +public class PercolateResponse extends BroadcastOperationResponse implements Iterable { - private List matches; + private long tookInMillis; + private Text[] matches; - PercolateResponse() { + public PercolateResponse(int totalShards, int successfulShards, int failedShards, List shardFailures, Text[] matches, long tookInMillis) { + super(totalShards, successfulShards, failedShards, shardFailures); + this.tookInMillis = tookInMillis; + this.matches = matches; + } + + public PercolateResponse(int totalShards, int successfulShards, int failedShards, List shardFailures, long tookInMillis) { + super(totalShards, successfulShards, failedShards, shardFailures); + this.tookInMillis = tookInMillis; + } + PercolateResponse() { } - public PercolateResponse(List matches) { + public PercolateResponse(Text[] matches) { this.matches = matches; } - public List getMatches() { + /** + * How long the percolate took. + */ + public TimeValue getTook() { + return new TimeValue(tookInMillis); + } + + /** + * How long the percolate took in milliseconds. + */ + public long getTookInMillis() { + return tookInMillis; + } + + public Text[] getMatches() { return this.matches; } @Override - public Iterator iterator() { - return matches.iterator(); + public Iterator iterator() { + return Arrays.asList(matches).iterator(); } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); - int size = in.readVInt(); - matches = new ArrayList(size); - for (int i = 0; i < size; i++) { - matches.add(in.readString()); - } + tookInMillis = in.readVLong(); + matches = in.readTextArray(); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeVInt(matches.size()); - for (String match : matches) { - out.writeString(match); - } + out.writeVLong(tookInMillis); + out.writeTextArray(matches); } } diff --git a/src/main/java/org/elasticsearch/action/percolate/PercolateShardRequest.java b/src/main/java/org/elasticsearch/action/percolate/PercolateShardRequest.java new file mode 100644 index 0000000000000..516551c9cb355 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/percolate/PercolateShardRequest.java @@ -0,0 +1,48 @@ +package org.elasticsearch.action.percolate; + +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationRequest; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + */ +public class PercolateShardRequest extends BroadcastShardOperationRequest { + + private String documentType; + private BytesReference documentSource; + + public PercolateShardRequest() { + } + + public PercolateShardRequest(String index, int shardId, PercolateRequest request) { + super(index, shardId, request); + this.documentType = request.documentType(); + this.documentSource = request.documentSource(); + } + + public String documentType() { + return documentType; + } + + public BytesReference documentSource() { + return documentSource; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + documentType = in.readString(); + documentSource = in.readBytesReference(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(documentType); + out.writeBytesReference(documentSource); + } + +} diff --git a/src/main/java/org/elasticsearch/action/percolate/PercolateShardResponse.java b/src/main/java/org/elasticsearch/action/percolate/PercolateShardResponse.java new file mode 100644 index 0000000000000..634ee0ebf6609 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/percolate/PercolateShardResponse.java @@ -0,0 +1,43 @@ +package org.elasticsearch.action.percolate; + +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.text.Text; + +import java.io.IOException; + +/** + */ +public class PercolateShardResponse extends BroadcastShardOperationResponse { + + private Text[] matches; + + public PercolateShardResponse() { + } + + public PercolateShardResponse(Text[] matches, String index, int shardId) { + super(index, shardId); + this.matches = matches; + } + + public Text[] matches() { + return matches; + } + + public void matches(Text[] matches) { + this.matches = matches; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + matches = in.readTextArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeTextArray(matches); + } +} diff --git a/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java b/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java index f23c6f044aff1..aaed8e862ab11 100644 --- a/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java +++ b/src/main/java/org/elasticsearch/action/percolate/TransportPercolateAction.java @@ -20,33 +20,51 @@ package org.elasticsearch.action.percolate; import org.elasticsearch.ElasticSearchException; -import org.elasticsearch.action.support.single.custom.TransportSingleCustomOperationAction; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.support.DefaultShardOperationFailedException; +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException; +import org.elasticsearch.action.support.broadcast.TransportBroadcastOperationAction; 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.ShardsIterator; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.percolator.PercolatorExecutor; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.index.percolator.PercolateException; import org.elasticsearch.index.percolator.PercolatorService; -import org.elasticsearch.index.service.IndexService; -import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.index.shard.ShardId; 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; + /** * */ -public class TransportPercolateAction extends TransportSingleCustomOperationAction { +public class TransportPercolateAction extends TransportBroadcastOperationAction { - private final IndicesService indicesService; + private final PercolatorService percolatorService; @Inject - public TransportPercolateAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, - IndicesService indicesService) { + public TransportPercolateAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, + TransportService transportService, PercolatorService percolatorService) { super(settings, threadPool, clusterService, transportService); - this.indicesService = indicesService; + this.percolatorService = percolatorService; + } + + @Override + protected void doExecute(PercolateRequest request, ActionListener listener) { + request.startTime = System.currentTimeMillis(); + super.doExecute(request, listener); } @Override @@ -59,11 +77,6 @@ protected PercolateRequest newRequest() { return new PercolateRequest(); } - @Override - protected PercolateResponse newResponse() { - return new PercolateResponse(); - } - @Override protected String transportAction() { return PercolateAction.NAME; @@ -75,22 +88,87 @@ protected ClusterBlockException checkGlobalBlock(ClusterState state, PercolateRe } @Override - protected ClusterBlockException checkRequestBlock(ClusterState state, PercolateRequest request) { - request.index(state.metaData().concreteIndex(request.index())); - return state.blocks().indexBlockedException(ClusterBlockLevel.READ, request.index()); + protected ClusterBlockException checkRequestBlock(ClusterState state, PercolateRequest request, String[] concreteIndices) { + return state.blocks().indicesBlockedException(ClusterBlockLevel.READ, concreteIndices); + } + + @Override + protected PercolateResponse newResponse(PercolateRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { + int successfulShards = 0; + int failedShards = 0; + + List shardResults = null; + List shardFailures = null; + + for (int i = 0; i < shardsResponses.length(); i++) { + Object shardResponse = shardsResponses.get(i); + if (shardResponse == null) { + failedShards++; + } else if (shardResponse instanceof BroadcastShardOperationFailedException) { + failedShards++; + if (shardFailures == null) { + shardFailures = newArrayList(); + } + shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse)); + } else { + PercolateShardResponse percolateShardResponse = (PercolateShardResponse) shardResponse; + if (shardResults == null) { + shardResults = newArrayList(); + } + shardResults.add(percolateShardResponse.matches()); + successfulShards++; + } + } + + long tookInMillis = System.currentTimeMillis() - request.startTime; + if (shardResults == null) { + return new PercolateResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures, tookInMillis); + } + + int size = 0; + for (Text[] shardResult : shardResults) { + size += shardResult.length; + } + Text[] finalMatches = new Text[size]; + int offset = 0; + for (Text[] shardResult : shardResults) { + System.arraycopy(shardResult, 0, finalMatches, offset, shardResult.length); + offset += shardResult.length; + } + assert size == offset; + return new PercolateResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures, finalMatches, tookInMillis); + } + + @Override + protected PercolateShardRequest newShardRequest() { + return new PercolateShardRequest(); } @Override - protected ShardsIterator shards(ClusterState clusterState, PercolateRequest request) { - return clusterState.routingTable().index(request.index()).randomAllActiveShardsIt(); + protected PercolateShardRequest newShardRequest(ShardRouting shard, PercolateRequest request) { + return new PercolateShardRequest(shard.index(), shard.id(), request); } @Override - protected PercolateResponse shardOperation(PercolateRequest request, int shardId) throws ElasticSearchException { - IndexService indexService = indicesService.indexServiceSafe(request.index()); - PercolatorService percolatorService = indexService.percolateService(); + protected PercolateShardResponse newShardResponse() { + return new PercolateShardResponse(); + } + + @Override + protected GroupShardsIterator shards(ClusterState clusterState, PercolateRequest request, String[] concreteIndices) { + Map> routingMap = clusterState.metaData().resolveSearchRouting(request.routing(), request.indices()); + return clusterService.operationRouting().searchShards(clusterState, request.indices(), concreteIndices, routingMap, request.preference()); + } - PercolatorExecutor.Response percolate = percolatorService.percolate(new PercolatorExecutor.SourceRequest(request.type(), request.source())); - return new PercolateResponse(percolate.matches()); + @Override + protected PercolateShardResponse shardOperation(PercolateShardRequest request) throws ElasticSearchException { + try { + return percolatorService.percolate(request); + } catch (Throwable t) { + logger.trace("[{}][{}] failed to percolate", t, request.index(), request.shardId()); + ShardId shardId = new ShardId(request.index(), request.shardId()); + throw new PercolateException(shardId, "failed to percolate", t); + } } + } diff --git a/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java b/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java index 53cdcf1328713..909a853b00dfe 100644 --- a/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java +++ b/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java @@ -193,9 +193,7 @@ protected void shardOperation(final UpdateRequest request, final ActionListener< indexAction.execute(upsertRequest, new ActionListener() { @Override public void onResponse(IndexResponse response) { - UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), - response.getVersion(), response.isCreated()); - update.setMatches(response.getMatches()); + UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); if (request.fields() != null && request.fields().length > 0) { Tuple> sourceAndContent = XContentHelper.convertToMap(upsertSourceBytes, true); update.setGetResult(updateHelper.extractGetResult(request, response.getVersion(), sourceAndContent.v2(), sourceAndContent.v1(), upsertSourceBytes)); @@ -230,9 +228,7 @@ public void run() { indexAction.execute(indexRequest, new ActionListener() { @Override public void onResponse(IndexResponse response) { - UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), - response.getVersion(), response.isCreated()); - update.setMatches(response.getMatches()); + UpdateResponse update = new UpdateResponse(response.getIndex(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); update.setGetResult(updateHelper.extractGetResult(request, response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); listener.onResponse(update); } diff --git a/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 476c32001373f..587312c1a5962 100644 --- a/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -76,7 +76,6 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { // it has to be a "create!" .create(true) .routing(request.routing()) - .percolate(request.percolate()) .refresh(request.refresh()) .replicationType(request.replicationType()).consistencyLevel(request.consistencyLevel()); indexRequest.operationThreaded(false); @@ -170,7 +169,6 @@ public Result prepare(UpdateRequest request, IndexShard indexShard) { .version(updateVersion).versionType(request.versionType()) .replicationType(request.replicationType()).consistencyLevel(request.consistencyLevel()) .timestamp(timestamp).ttl(ttl) - .percolate(request.percolate()) .refresh(request.refresh()); indexRequest.operationThreaded(false); return new Result(indexRequest, Operation.INDEX, updatedSourceAsMap, updateSourceContentType); diff --git a/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index 8a70674ac92b0..d0802655c5ef3 100644 --- a/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -65,8 +65,6 @@ public class UpdateRequest extends InstanceShardOperationRequest private VersionType versionType = VersionType.INTERNAL; private int retryOnConflict = 0; - private String percolate; - private boolean refresh = false; private ReplicationType replicationType = ReplicationType.DEFAULT; @@ -319,20 +317,6 @@ public VersionType versionType() { return this.versionType; } - /** - * Causes the update request document to be percolated. The parameter is the percolate query - * to use to reduce the percolated queries that are going to run against this doc. Can be - * set to * to indicate that all percolate queries should be run. - */ - public UpdateRequest percolate(String percolate) { - this.percolate = percolate; - return this; - } - - public String percolate() { - return this.percolate; - } - /** * Should a refresh be executed post this update operation causing the operation to * be searchable. Note, heavy indexing should not set this to true. Defaults @@ -603,7 +587,6 @@ public void readFrom(StreamInput in) throws IOException { scriptLang = in.readOptionalString(); scriptParams = in.readMap(); retryOnConflict = in.readVInt(); - percolate = in.readOptionalString(); refresh = in.readBoolean(); if (in.readBoolean()) { doc = new IndexRequest(); @@ -639,7 +622,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(scriptLang); out.writeMap(scriptParams); out.writeVInt(retryOnConflict); - out.writeOptionalString(percolate); out.writeBoolean(refresh); if (doc == null) { out.writeBoolean(false); diff --git a/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java b/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java index bc3b02046c606..aab5f6f18b087 100644 --- a/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java +++ b/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java @@ -169,16 +169,6 @@ public UpdateRequestBuilder setConsistencyLevel(WriteConsistencyLevel consistenc return this; } - /** - * Causes the updated document to be percolated. The parameter is the percolate query - * to use to reduce the percolated queries that are going to run against this doc. Can be - * set to * to indicate that all percolate queries should be run. - */ - public UpdateRequestBuilder setPercolate(String percolate) { - request.percolate(percolate); - return this; - } - /** * Sets the doc to use for updates when a script is not specified. */ diff --git a/src/main/java/org/elasticsearch/action/update/UpdateResponse.java b/src/main/java/org/elasticsearch/action/update/UpdateResponse.java index 6c8ffff54f015..215986e697fad 100644 --- a/src/main/java/org/elasticsearch/action/update/UpdateResponse.java +++ b/src/main/java/org/elasticsearch/action/update/UpdateResponse.java @@ -19,15 +19,12 @@ package org.elasticsearch.action.update; -import com.google.common.collect.ImmutableList; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.get.GetResult; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; /** */ @@ -38,7 +35,6 @@ public class UpdateResponse extends ActionResponse { private String type; private long version; private boolean created; - private List matches; private GetResult getResult; public UpdateResponse() { @@ -81,13 +77,6 @@ public long getVersion() { return this.version; } - /** - * Returns the percolate queries matches. null if no percolation was requested. - */ - public List getMatches() { - return this.matches; - } - public void setGetResult(GetResult getResult) { this.getResult = getResult; } @@ -96,7 +85,6 @@ public GetResult getGetResult() { return this.getResult; } - /** * Returns true if document was created due to an UPSERT operation */ @@ -104,12 +92,6 @@ public boolean isCreated() { return this.created; } - /** - * Internal. - */ - public void setMatches(List matches) { - this.matches = matches; - } @Override public void readFrom(StreamInput in) throws IOException { @@ -119,27 +101,6 @@ public void readFrom(StreamInput in) throws IOException { type = in.readString(); version = in.readLong(); created = in.readBoolean(); - if (in.readBoolean()) { - int size = in.readVInt(); - if (size == 0) { - matches = ImmutableList.of(); - } else if (size == 1) { - matches = ImmutableList.of(in.readString()); - } else if (size == 2) { - matches = ImmutableList.of(in.readString(), in.readString()); - } else if (size == 3) { - matches = ImmutableList.of(in.readString(), in.readString(), in.readString()); - } else if (size == 4) { - matches = ImmutableList.of(in.readString(), in.readString(), in.readString(), in.readString()); - } else if (size == 5) { - matches = ImmutableList.of(in.readString(), in.readString(), in.readString(), in.readString(), in.readString()); - } else { - matches = new ArrayList(); - for (int i = 0; i < size; i++) { - matches.add(in.readString()); - } - } - } if (in.readBoolean()) { getResult = GetResult.readGetResult(in); } @@ -153,15 +114,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(type); out.writeLong(version); out.writeBoolean(created); - if (matches == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - out.writeVInt(matches.size()); - for (String match : matches) { - out.writeString(match); - } - } if (getResult == null) { out.writeBoolean(false); } else { diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 68747ea037608..459bc7211ddb4 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -54,7 +54,6 @@ import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.index.service.IndexService; import org.elasticsearch.indices.IndexAlreadyExistsException; import org.elasticsearch.indices.IndicesService; @@ -157,20 +156,6 @@ public ClusterState execute(ClusterState currentState) { // add the request mapping Map> mappings = Maps.newHashMap(); - // if its a _percolator index, don't index the query object - if (request.index.equals(PercolatorService.INDEX_NAME)) { - mappings.put(MapperService.DEFAULT_MAPPING, parseMapping("{\n" + - " \"_default_\":{\n" + - " \"properties\" : {\n" + - " \"query\" : {\n" + - " \"type\" : \"object\",\n" + - " \"enabled\" : false\n" + - " }\n" + - " }\n" + - " }\n" + - "}")); - } - for (Map.Entry entry : request.mappings.entrySet()) { mappings.put(entry.getKey(), parseMapping(entry.getValue())); } @@ -226,29 +211,18 @@ public ClusterState execute(ClusterState currentState) { // now, put the request settings, so they override templates indexSettingsBuilder.put(request.settings); - if (request.index.equals(PercolatorService.INDEX_NAME)) { - // if its percolator, always 1 shard - indexSettingsBuilder.put(SETTING_NUMBER_OF_SHARDS, 1); - } else { - if (indexSettingsBuilder.get(SETTING_NUMBER_OF_SHARDS) == null) { - if (request.index.equals(riverIndexName)) { - indexSettingsBuilder.put(SETTING_NUMBER_OF_SHARDS, settings.getAsInt(SETTING_NUMBER_OF_SHARDS, 1)); - } else { - indexSettingsBuilder.put(SETTING_NUMBER_OF_SHARDS, settings.getAsInt(SETTING_NUMBER_OF_SHARDS, 5)); - } + if (indexSettingsBuilder.get(SETTING_NUMBER_OF_SHARDS) == null) { + if (request.index.equals(riverIndexName)) { + indexSettingsBuilder.put(SETTING_NUMBER_OF_SHARDS, settings.getAsInt(SETTING_NUMBER_OF_SHARDS, 1)); + } else { + indexSettingsBuilder.put(SETTING_NUMBER_OF_SHARDS, settings.getAsInt(SETTING_NUMBER_OF_SHARDS, 5)); } } - if (request.index.equals(PercolatorService.INDEX_NAME)) { - // if its percolator, always set number of replicas to 0, and expand to 0-all - indexSettingsBuilder.put(SETTING_NUMBER_OF_REPLICAS, 0); - indexSettingsBuilder.put(SETTING_AUTO_EXPAND_REPLICAS, "0-all"); - } else { - if (indexSettingsBuilder.get(SETTING_NUMBER_OF_REPLICAS) == null) { - if (request.index.equals(riverIndexName)) { - indexSettingsBuilder.put(SETTING_NUMBER_OF_REPLICAS, settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, 1)); - } else { - indexSettingsBuilder.put(SETTING_NUMBER_OF_REPLICAS, settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, 1)); - } + if (indexSettingsBuilder.get(SETTING_NUMBER_OF_REPLICAS) == null) { + if (request.index.equals(riverIndexName)) { + indexSettingsBuilder.put(SETTING_NUMBER_OF_REPLICAS, settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, 1)); + } else { + indexSettingsBuilder.put(SETTING_NUMBER_OF_REPLICAS, settings.getAsInt(SETTING_NUMBER_OF_REPLICAS, 1)); } } @@ -504,7 +478,7 @@ private void validate(Request request, ClusterState state) throws ElasticSearchE if (request.index.contains("#")) { throw new InvalidIndexNameException(new Index(request.index), request.index, "must not contain '#"); } - if (!request.index.equals(riverIndexName) && !request.index.equals(PercolatorService.INDEX_NAME) && request.index.charAt(0) == '_') { + if (!request.index.equals(riverIndexName) && request.index.charAt(0) == '_') { throw new InvalidIndexNameException(new Index(request.index), request.index, "must not start with '_'"); } if (!request.index.toLowerCase(Locale.ROOT).equals(request.index)) { diff --git a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index 6c32a98c059eb..67b64df310ee4 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -38,6 +38,7 @@ import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MergeMappingException; +import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.index.service.IndexService; import org.elasticsearch.indices.IndexMissingException; import org.elasticsearch.indices.IndicesService; @@ -330,7 +331,7 @@ public ClusterState execute(ClusterState currentState) { } else if (!mappingType.equals(newMappers.values().iterator().next().type())) { throw new InvalidTypeNameException("Type name provided does not match type name within mapping definition"); } - if (!MapperService.DEFAULT_MAPPING.equals(mappingType) && mappingType.charAt(0) == '_') { + if (!MapperService.DEFAULT_MAPPING.equals(mappingType) && !PercolatorService.Constants.TYPE_NAME.equals(mappingType) && mappingType.charAt(0) == '_') { throw new InvalidTypeNameException("Document mapping type name can't start with '_'"); } diff --git a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java index cc29ff6325d49..f7cbd65de522b 100644 --- a/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java +++ b/src/main/java/org/elasticsearch/common/io/stream/StreamInput.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.text.StringAndBytesText; +import org.elasticsearch.common.text.StringText; import org.elasticsearch.common.text.Text; import org.joda.time.DateTime; @@ -220,6 +221,18 @@ public Text readText() throws IOException { return new StringAndBytesText(readBytesReference(length)); } + public Text[] readTextArray() throws IOException { + int size = readVInt(); + if (size == 0) { + return StringText.EMPTY_ARRAY; + } + Text[] ret = new Text[size]; + for (int i = 0; i < size; i++) { + ret[i] = readText(); + } + return ret; + } + public Text readSharedText() throws IOException { return readText(); } diff --git a/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java b/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java index 1997d22a5f4f2..ff66e149f69df 100644 --- a/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java +++ b/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java @@ -204,6 +204,13 @@ public void writeText(Text text) throws IOException { } } + public void writeTextArray(Text[] array) throws IOException { + writeVInt(array.length); + for (Text t : array) { + writeText(t); + } + } + public void writeSharedText(Text text) throws IOException { writeText(text); } diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolateIndexUnavailable.java b/src/main/java/org/elasticsearch/index/mapper/DocumentTypeListener.java similarity index 52% rename from src/main/java/org/elasticsearch/index/percolator/PercolateIndexUnavailable.java rename to src/main/java/org/elasticsearch/index/mapper/DocumentTypeListener.java index 935ca22918ced..0db864282c4bb 100644 --- a/src/main/java/org/elasticsearch/index/percolator/PercolateIndexUnavailable.java +++ b/src/main/java/org/elasticsearch/index/mapper/DocumentTypeListener.java @@ -1,13 +1,13 @@ /* - * Licensed to Elastic Search and Shay Banon under one + * Licensed to ElasticSearch and Shay Banon under one * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information - * regarding copyright ownership. Elastic Search licenses this + * 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 + * 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 @@ -17,16 +17,24 @@ * under the License. */ -package org.elasticsearch.index.percolator; - -import org.elasticsearch.index.Index; +package org.elasticsearch.index.mapper; /** - * */ -public class PercolateIndexUnavailable extends PercolatorException { +public interface DocumentTypeListener { + + /** + * Invoked when a new document type has been created. + * + * @param type The document type that has been created + */ + void created(String type); + + /** + * Invoked when an existing document type has been removed. + * + * @param type The document type that has been removed + */ + void removed(String type); - public PercolateIndexUnavailable(Index index) { - super(index, "percolator index not allocated on this node"); - } } diff --git a/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 0b57c1e14ed7e..3fd382c1d8c2e 100644 --- a/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -33,6 +33,8 @@ import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.lucene.search.AndFilter; +import org.elasticsearch.common.lucene.search.NotFilter; import org.elasticsearch.common.lucene.search.TermFilter; import org.elasticsearch.common.lucene.search.XBooleanFilter; import org.elasticsearch.common.regex.Regex; @@ -45,6 +47,7 @@ import org.elasticsearch.index.codec.postingsformat.PostingsFormatService; import org.elasticsearch.index.mapper.internal.TypeFieldMapper; import org.elasticsearch.index.mapper.object.ObjectMapper; +import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.index.search.nested.NonNestedDocsFilter; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.similarity.SimilarityLookupService; @@ -56,10 +59,8 @@ import java.io.InputStreamReader; import java.net.MalformedURLException; import java.net.URL; -import java.util.Arrays; -import java.util.Collection; -import java.util.Map; -import java.util.Set; +import java.util.*; +import java.util.concurrent.CopyOnWriteArrayList; import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder; import static org.elasticsearch.index.mapper.DocumentMapper.MergeFlags.mergeFlags; @@ -72,7 +73,6 @@ public class MapperService extends AbstractIndexComponent implements Iterable mappers = ImmutableMap.of(); @@ -99,12 +100,13 @@ public class MapperService extends AbstractIndexComponent implements Iterable typeListeners = new CopyOnWriteArrayList(); + @Inject public MapperService(Index index, @IndexSettings Settings indexSettings, Environment environment, AnalysisService analysisService, PostingsFormatService postingsFormatService, SimilarityLookupService similarityLookupService) { super(index, indexSettings); this.analysisService = analysisService; - this.postingsFormatService = postingsFormatService; this.documentParser = new DocumentMapperParser(index, indexSettings, analysisService, postingsFormatService, similarityLookupService); this.searchAnalyzer = new SmartIndexNameSearchAnalyzer(analysisService.defaultSearchAnalyzer()); this.searchQuoteAnalyzer = new SmartIndexNameSearchQuoteAnalyzer(analysisService.defaultSearchQuoteAnalyzer()); @@ -149,7 +151,40 @@ public MapperService(Index index, @IndexSettings Settings indexSettings, Environ } } - logger.debug("using dynamic[{}], default mapping: default_mapping_location[{}], loaded_from[{}] and source[{}]", dynamic, defaultMappingLocation, defaultMappingUrl, defaultMappingSource); + String percolatorMappingLocation = componentSettings.get("percolator_mapping_location"); + URL percolatorMappingUrl = null; + if (percolatorMappingLocation != null) { + try { + percolatorMappingUrl = environment.resolveConfig(percolatorMappingLocation); + } catch (FailedToResolveConfigException e) { + // not there, default to the built in one + try { + percolatorMappingUrl = new File(percolatorMappingLocation).toURI().toURL(); + } catch (MalformedURLException e1) { + throw new FailedToResolveConfigException("Failed to resolve percolator mapping location [" + defaultMappingLocation + "]"); + } + } + } + if (percolatorMappingUrl != null) { + try { + percolatorMappingSource = Streams.copyToString(new InputStreamReader(percolatorMappingUrl.openStream(), Charsets.UTF_8)); + } catch (IOException e) { + throw new MapperException("Failed to load default percolator mapping source from [" + percolatorMappingUrl + "]", e); + } + } else { + percolatorMappingSource = "{\n" + + " \"_percolator\":{\n" + + " \"properties\" : {\n" + + " \"query\" : {\n" + + " \"type\" : \"object\",\n" + + " \"enabled\" : false\n" + + " }\n" + + " }\n" + + " }\n" + + "}"; + } + + logger.debug("using dynamic[{}], default mapping: default_mapping_location[{}], loaded_from[{}] and source[{}], default percolator mapping: location[{}], loaded_from[{}] and source[{}]", dynamic, defaultMappingLocation, defaultMappingUrl, defaultMappingSource, percolatorMappingLocation, percolatorMappingUrl, percolatorMappingSource); } public void close() { @@ -175,6 +210,14 @@ public DocumentMapperParser documentMapperParser() { return this.documentParser; } + public void addTypeListener(DocumentTypeListener listener) { + typeListeners.add(listener); + } + + public void removeTypeListener(DocumentTypeListener listener) { + typeListeners.remove(listener); + } + public DocumentMapper merge(String type, String mappingSource, boolean applyDefault) { if (DEFAULT_MAPPING.equals(type)) { // verify we can parse it @@ -198,7 +241,7 @@ private DocumentMapper merge(DocumentMapper mapper) { if (mapper.type().length() == 0) { throw new InvalidTypeNameException("mapping type name is empty"); } - if (mapper.type().charAt(0) == '_') { + if (mapper.type().charAt(0) == '_' && !PercolatorService.Constants.TYPE_NAME.equals(mapper.type())) { throw new InvalidTypeNameException("mapping type name [" + mapper.type() + "] can't start with '_'"); } if (mapper.type().contains("#")) { @@ -236,6 +279,9 @@ private DocumentMapper merge(DocumentMapper mapper) { mapper.addObjectMapperListener(objectMapperListener, false); mappers = newMapBuilder(mappers).put(mapper.type(), mapper).map(); + for (DocumentTypeListener typeListener : typeListeners) { + typeListener.created(mapper.type()); + } return mapper; } } @@ -309,6 +355,9 @@ public void remove(String type) { docMapper.close(); mappers = newMapBuilder(mappers).remove(type).map(); removeObjectAndFieldMappers(docMapper); + for (DocumentTypeListener typeListener : typeListeners) { + typeListener.removed(type); + } } } @@ -377,6 +426,12 @@ public DocumentMapper parse(String mappingType, String mappingSource) throws Map } public DocumentMapper parse(String mappingType, String mappingSource, boolean applyDefault) throws MapperParsingException { + String defaultMappingSource; + if (PercolatorService.Constants.TYPE_NAME.equals(mappingType)) { + defaultMappingSource = percolatorMappingSource; + } else { + defaultMappingSource = this.defaultMappingSource; + } return documentParser.parse(mappingType, mappingSource, applyDefault ? defaultMappingSource : null); } @@ -416,9 +471,27 @@ public DocumentMapper documentMapperWithAutoCreate(String type) { */ @Nullable public Filter searchFilter(String... types) { + boolean filterPercolateType = hasMapping(PercolatorService.Constants.TYPE_NAME); + if (types != null && filterPercolateType) { + for (String type : types) { + if (PercolatorService.Constants.TYPE_NAME.equals(type)) { + filterPercolateType = false; + break; + } + } + } + Filter excludePercolatorType = null; + if (filterPercolateType) { + excludePercolatorType = new NotFilter(documentMapper(PercolatorService.Constants.TYPE_NAME).typeFilter()); + } + if (types == null || types.length == 0) { - if (hasNested) { + if (hasNested && filterPercolateType) { + return new AndFilter(ImmutableList.of(excludePercolatorType, NonNestedDocsFilter.INSTANCE)); + } else if (hasNested) { return NonNestedDocsFilter.INSTANCE; + } else if (filterPercolateType) { + return excludePercolatorType; } else { return null; } @@ -445,13 +518,20 @@ public Filter searchFilter(String... types) { break; } } + if (useTermsFilter) { BytesRef[] typesBytes = new BytesRef[types.length]; for (int i = 0; i < typesBytes.length; i++) { typesBytes[i] = new BytesRef(types[i]); } - return new TermsFilter(TypeFieldMapper.NAME, typesBytes); + TermsFilter termsFilter = new TermsFilter(TypeFieldMapper.NAME, typesBytes); + if (filterPercolateType) { + return new AndFilter(ImmutableList.of(excludePercolatorType, termsFilter)); + } else { + return termsFilter; + } } else { + // Current bool filter requires that at least one should clause matches, even with a must clause. XBooleanFilter bool = new XBooleanFilter(); for (String type : types) { DocumentMapper docMapper = documentMapper(type); @@ -461,6 +541,10 @@ public Filter searchFilter(String... types) { bool.add(new FilterClause(docMapper.typeFilter(), BooleanClause.Occur.SHOULD)); } } + if (filterPercolateType) { + bool.add(excludePercolatorType, BooleanClause.Occur.MUST); + } + return bool; } } diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolateException.java b/src/main/java/org/elasticsearch/index/percolator/PercolateException.java new file mode 100644 index 0000000000000..06e14b0aff810 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/percolator/PercolateException.java @@ -0,0 +1,26 @@ +package org.elasticsearch.index.percolator; + +import org.elasticsearch.ElasticSearchException; +import org.elasticsearch.index.shard.ShardId; + +/** + * Exception during percolating document(s) at runtime. + */ +public class PercolateException extends ElasticSearchException { + + private final ShardId shardId; + + public PercolateException(String msg, ShardId shardId) { + super(msg); + this.shardId = shardId; + } + + public PercolateException(ShardId shardId, String msg, Throwable cause) { + super(msg, cause); + this.shardId = shardId; + } + + public ShardId getShardId() { + return shardId; + } +} diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolatorException.java b/src/main/java/org/elasticsearch/index/percolator/PercolatorException.java index ebc345208dff1..b3a7783dd479c 100644 --- a/src/main/java/org/elasticsearch/index/percolator/PercolatorException.java +++ b/src/main/java/org/elasticsearch/index/percolator/PercolatorException.java @@ -23,7 +23,7 @@ import org.elasticsearch.index.IndexException; /** - * + * Exception during indexing a percolator query. */ public class PercolatorException extends IndexException { diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolatorExecutor.java b/src/main/java/org/elasticsearch/index/percolator/PercolatorExecutor.java deleted file mode 100644 index 653158df71b44..0000000000000 --- a/src/main/java/org/elasticsearch/index/percolator/PercolatorExecutor.java +++ /dev/null @@ -1,575 +0,0 @@ -/* - * Licensed to Elastic Search and Shay Banon under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Elastic Search licenses this - * file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.percolator; - -import org.apache.lucene.analysis.TokenStream; -import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.index.IndexableField; -import org.apache.lucene.index.memory.MemoryIndex; -import org.apache.lucene.index.memory.ReusableMemoryIndex; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.IndexSearcher; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.Scorer; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.ElasticSearchException; -import org.elasticsearch.ElasticSearchIllegalArgumentException; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Preconditions; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.logging.ESLogger; -import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.settings.ImmutableSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.common.xcontent.XContentHelper; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.AbstractIndexComponent; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.cache.IndexCache; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.fielddata.BytesValues; -import org.elasticsearch.index.fielddata.FieldDataType; -import org.elasticsearch.index.fielddata.IndexFieldData; -import org.elasticsearch.index.fielddata.IndexFieldDataService; -import org.elasticsearch.index.mapper.*; -import org.elasticsearch.index.mapper.internal.UidFieldMapper; -import org.elasticsearch.index.query.IndexQueryParserService; -import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.index.service.IndexService; -import org.elasticsearch.index.settings.IndexSettings; -import org.elasticsearch.index.settings.IndexSettingsService; -import org.elasticsearch.index.shard.service.IndexShard; -import org.elasticsearch.indices.IndicesService; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Queue; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.elasticsearch.index.mapper.SourceToParse.source; - -/** - * - */ -public class PercolatorExecutor extends AbstractIndexComponent { - - public static class SourceRequest { - private final String type; - private final BytesReference source; - - public SourceRequest(String type, BytesReference source) { - this.type = type; - this.source = source; - } - - public String type() { - return this.type; - } - - public BytesReference source() { - return source; - } - } - - public static class DocAndSourceQueryRequest { - private final ParsedDocument doc; - @Nullable - private final String query; - - public DocAndSourceQueryRequest(ParsedDocument doc, @Nullable String query) { - this.doc = doc; - this.query = query; - } - - public ParsedDocument doc() { - return this.doc; - } - - @Nullable - String query() { - return this.query; - } - } - - - public static class DocAndQueryRequest { - private final ParsedDocument doc; - @Nullable - private final Query query; - - public DocAndQueryRequest(ParsedDocument doc, @Nullable Query query) { - this.doc = doc; - this.query = query; - } - - public ParsedDocument doc() { - return this.doc; - } - - @Nullable - Query query() { - return this.query; - } - } - - public static final class Response { - private final List matches; - private final boolean mappersAdded; - - public Response(List matches, boolean mappersAdded) { - this.matches = matches; - this.mappersAdded = mappersAdded; - } - - public boolean mappersAdded() { - return this.mappersAdded; - } - - public List matches() { - return matches; - } - } - - private final MapperService mapperService; - - private final IndexQueryParserService queryParserService; - - private final IndexCache indexCache; - private final IndexFieldDataService fieldDataService; - - private final Map queries = ConcurrentCollections.newConcurrentMap(); - - /** - * Realtime index setting to control the number of MemoryIndex instances used to handle - * Percolate requests. The default is 10 - */ - public static final String PERCOLATE_POOL_SIZE = "index.percolate.pool.size"; - - /** - * Realtime index setting to control the upper memory reuse limit across all {@link MemoryIndex} instances - * pooled to handle Percolate requests. This is NOT a peak upper bound, percolate requests can use more memory than this upper - * bound. Yet, if all pooled {@link MemoryIndex} instances are returned to the pool this marks the upper memory bound use - * buy this idle instances. If more memory was allocated by a {@link MemoryIndex} the additinal memory is freed before it - * returns to the pool. The default is 1 MB - */ - public static final String PERCOLATE_POOL_MAX_MEMORY = "index.percolate.pool.reuse_memory_size"; - - /** - * Realtime index setting to control the timeout or the maximum waiting time - * for an pooled memory index until an extra memory index is created. The default is 100 ms - */ - public static final String PERCOLATE_TIMEOUT = "index.percolate.pool.timeout"; - - /** - * Simple {@link MemoryIndex} Pool that reuses MemoryIndex instance across threads and allows each of the - * MemoryIndex instance to reuse its internal memory based on a user configured realtime value. - */ - static final class MemoryIndexPool { - private volatile BlockingQueue memoryIndexQueue; - - // used to track the in-flight memoryIdx instances so we don't overallocate - private int poolMaxSize; - private int poolCurrentSize; - private volatile long bytesPerMemoryIndex; - private ByteSizeValue maxMemorySize; // only accessed in sync block - private volatile TimeValue timeout; - public MemoryIndexPool(Settings settings) { - poolMaxSize = settings.getAsInt(PERCOLATE_POOL_SIZE, 10); - if (poolMaxSize <= 0) { - throw new ElasticSearchIllegalArgumentException(PERCOLATE_POOL_SIZE + " size must be > 0 but was [" + poolMaxSize + "]"); - } - memoryIndexQueue = new ArrayBlockingQueue(poolMaxSize); - maxMemorySize = settings.getAsBytesSize(PERCOLATE_POOL_MAX_MEMORY, new ByteSizeValue(1, ByteSizeUnit.MB)); - if (maxMemorySize.bytes() < 0) { - throw new ElasticSearchIllegalArgumentException(PERCOLATE_POOL_MAX_MEMORY + " must be positive but was [" + maxMemorySize.bytes() + "]"); - } - timeout = settings.getAsTime(PERCOLATE_TIMEOUT, new TimeValue(100)); - if (timeout.millis() < 0) { - throw new ElasticSearchIllegalArgumentException(PERCOLATE_TIMEOUT + " must be positive but was [" + timeout + "]"); - } - bytesPerMemoryIndex = maxMemorySize.bytes() / poolMaxSize; - } - - public synchronized void updateSettings(Settings settings) { - - final int newPoolSize = settings.getAsInt(PERCOLATE_POOL_SIZE, poolMaxSize); - if (newPoolSize <= 0) { - throw new ElasticSearchIllegalArgumentException(PERCOLATE_POOL_SIZE + " size must be > 0 but was [" + newPoolSize + "]"); - } - final ByteSizeValue byteSize = settings.getAsBytesSize(PERCOLATE_POOL_MAX_MEMORY, maxMemorySize); - if (byteSize.bytes() < 0) { - throw new ElasticSearchIllegalArgumentException(PERCOLATE_POOL_MAX_MEMORY + " must be positive but was [" + byteSize.bytes() + "]"); - } - timeout = settings.getAsTime(PERCOLATE_TIMEOUT, timeout); // always set this! - if (timeout.millis() < 0) { - throw new ElasticSearchIllegalArgumentException(PERCOLATE_TIMEOUT + " must be positive but was [" + timeout + "]"); - } - if (maxMemorySize.equals(byteSize) && newPoolSize == poolMaxSize) { - // nothing changed - return - return; - } - maxMemorySize = byteSize; - poolMaxSize = newPoolSize; - poolCurrentSize = Integer.MAX_VALUE; // prevent new creations until we have the new index in place - /* - * if this has changed we simply change the blocking queue instance with a new pool - * size and reset the - */ - bytesPerMemoryIndex = byteSize.bytes() / newPoolSize; - memoryIndexQueue = new ArrayBlockingQueue(newPoolSize); - poolCurrentSize = 0; // lets refill the queue - } - - public ReusableMemoryIndex acquire() { - final BlockingQueue queue = memoryIndexQueue; - final ReusableMemoryIndex poll = queue.poll(); - return poll == null ? waitOrCreate(queue) : poll; - } - - private ReusableMemoryIndex waitOrCreate(BlockingQueue queue) { - synchronized (this) { - if (poolCurrentSize < poolMaxSize) { - poolCurrentSize++; - return new ReusableMemoryIndex(false, bytesPerMemoryIndex); - - } - } - ReusableMemoryIndex poll = null; - try { - final TimeValue timeout = this.timeout; // only read the volatile var once - poll = queue.poll(timeout.getMillis(), TimeUnit.MILLISECONDS); // delay this by 100ms by default - } catch (InterruptedException ie) { - // don't swallow the interrupt - Thread.currentThread().interrupt(); - } - return poll == null ? new ReusableMemoryIndex(false, bytesPerMemoryIndex) : poll; - } - - public void release(ReusableMemoryIndex index) { - assert index != null : "can't release null reference"; - if (bytesPerMemoryIndex == index.getMaxReuseBytes()) { - index.reset(); - // only put is back into the queue if the size fits - prune old settings on the fly - memoryIndexQueue.offer(index); - } - } - - } - - - private IndicesService indicesService; - private final MemoryIndexPool memIndexPool; - - @Inject - public PercolatorExecutor(Index index, @IndexSettings Settings indexSettings, - MapperService mapperService, IndexQueryParserService queryParserService, - IndexCache indexCache, IndexFieldDataService fieldDataService, IndexSettingsService indexSettingsService) { - super(index, indexSettings); - this.mapperService = mapperService; - this.queryParserService = queryParserService; - this.indexCache = indexCache; - this.fieldDataService = fieldDataService; - memIndexPool = new MemoryIndexPool(indexSettings); - ApplySettings applySettings = new ApplySettings(); - indexSettingsService.addListener(applySettings); - } - - class ApplySettings implements IndexSettingsService.Listener { - @Override - public void onRefreshSettings(Settings settings) { - memIndexPool.updateSettings(settings); - } - } - - public void setIndicesService(IndicesService indicesService) { - this.indicesService = indicesService; - } - - public void close() { - this.queries.clear(); - } - - public void addQuery(String name, QueryBuilder queryBuilder) throws ElasticSearchException { - try { - XContentBuilder builder = XContentFactory.smileBuilder() - .startObject().field("query", queryBuilder).endObject(); - addQuery(name, builder.bytes()); - } catch (IOException e) { - throw new ElasticSearchException("Failed to add query [" + name + "]", e); - } - } - - public void addQuery(String name, BytesReference source) throws ElasticSearchException { - addQuery(name, parseQuery(name, source)); - } - - public Query parseQuery(String name, BytesReference source) throws ElasticSearchException { - XContentParser parser = null; - try { - parser = XContentHelper.createParser(source); - Query query = null; - String currentFieldName = null; - XContentParser.Token token = parser.nextToken(); // move the START_OBJECT - if (token != XContentParser.Token.START_OBJECT) { - throw new ElasticSearchException("failed to parse query [" + name + "], not starting with OBJECT"); - } - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else if (token == XContentParser.Token.START_OBJECT) { - if ("query".equals(currentFieldName)) { - query = queryParserService.parse(parser).query(); - break; - } else { - parser.skipChildren(); - } - } else if (token == XContentParser.Token.START_ARRAY) { - parser.skipChildren(); - } - } - return query; - } catch (Exception e) { - throw new ElasticSearchException("failed to parse query [" + name + "]", e); - } finally { - if (parser != null) { - parser.close(); - } - } - } - - private void addQuery(String name, Query query) { - Preconditions.checkArgument(query != null, "query must be provided for percolate request"); - this.queries.put(name, query); - } - - public void removeQuery(String name) { - this.queries.remove(name); - } - - public void addQueries(Map queries) { - this.queries.putAll(queries); - } - - public Response percolate(final SourceRequest request) throws ElasticSearchException { - Query query = null; - ParsedDocument doc = null; - XContentParser parser = null; - try { - - parser = XContentFactory.xContent(request.source()).createParser(request.source()); - String currentFieldName = null; - XContentParser.Token token; - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - // we need to check the "doc" here, so the next token will be START_OBJECT which is - // the actual document starting - if ("doc".equals(currentFieldName)) { - DocumentMapper docMapper = mapperService.documentMapperWithAutoCreate(request.type()); - doc = docMapper.parse(source(parser).type(request.type()).flyweight(true)); - } - } else if (token == XContentParser.Token.START_OBJECT) { - if ("query".equals(currentFieldName)) { - query = percolatorIndexServiceSafe().queryParserService().parse(parser).query(); - } - } else if (token == null) { - break; - } - } - } catch (IOException e) { - throw new PercolatorException(index, "failed to parse request", e); - } finally { - if (parser != null) { - parser.close(); - } - } - - if (doc == null) { - throw new PercolatorException(index, "No doc to percolate in the request"); - } - - return percolate(new DocAndQueryRequest(doc, query)); - } - - public Response percolate(DocAndSourceQueryRequest request) throws ElasticSearchException { - Query query = null; - if (Strings.hasLength(request.query()) && !request.query().equals("*")) { - query = percolatorIndexServiceSafe().queryParserService().parse(QueryBuilders.queryString(request.query())).query(); - } - return percolate(new DocAndQueryRequest(request.doc(), query)); - } - - private Response percolate(DocAndQueryRequest request) throws ElasticSearchException { - // first, parse the source doc into a MemoryIndex - final ReusableMemoryIndex memoryIndex = memIndexPool.acquire(); - try { - // TODO: This means percolation does not support nested docs... - for (IndexableField field : request.doc().rootDoc().getFields()) { - if (!field.fieldType().indexed()) { - continue; - } - // no need to index the UID field - if (field.name().equals(UidFieldMapper.NAME)) { - continue; - } - TokenStream tokenStream; - try { - tokenStream = field.tokenStream(request.doc().analyzer()); - if (tokenStream != null) { - memoryIndex.addField(field.name(), tokenStream, field.boost()); - } - } catch (IOException e) { - throw new ElasticSearchException("Failed to create token stream", e); - } - } - - final IndexSearcher searcher = memoryIndex.createSearcher(); - List matches = new ArrayList(); - - try { - if (request.query() == null) { - Lucene.ExistsCollector collector = new Lucene.ExistsCollector(); - for (Map.Entry entry : queries.entrySet()) { - collector.reset(); - try { - searcher.search(entry.getValue(), collector); - } catch (IOException e) { - logger.warn("[" + entry.getKey() + "] failed to execute query", e); - } - - if (collector.exists()) { - matches.add(entry.getKey()); - } - } - } else { - IndexService percolatorIndex = percolatorIndexServiceSafe(); - if (percolatorIndex.numberOfShards() == 0) { - throw new PercolateIndexUnavailable(new Index(PercolatorService.INDEX_NAME)); - } - IndexShard percolatorShard = percolatorIndex.shard(0); - Engine.Searcher percolatorSearcher = percolatorShard.searcher(); - try { - percolatorSearcher.searcher().search(request.query(), new QueryCollector(logger, queries, searcher, percolatorIndex, matches)); - } catch (IOException e) { - logger.warn("failed to execute", e); - } finally { - percolatorSearcher.release(); - } - } - } finally { - // explicitly clear the reader, since we can only register on callback on SegmentReader - indexCache.clear(searcher.getIndexReader()); - fieldDataService.clear(searcher.getIndexReader()); - } - return new Response(matches, request.doc().mappingsModified()); - } finally { - memIndexPool.release(memoryIndex); - } - - } - - private IndexService percolatorIndexServiceSafe() { - IndexService indexService = indicesService.indexService(PercolatorService.INDEX_NAME); - if (indexService == null) { - throw new PercolateIndexUnavailable(new Index(PercolatorService.INDEX_NAME)); - } - return indexService; - } - - static class QueryCollector extends Collector { - private final IndexFieldData uidFieldData; - private final IndexSearcher searcher; - private final IndexService percolatorIndex; - private final List matches; - private final Map queries; - private final ESLogger logger; - - private final Lucene.ExistsCollector collector = new Lucene.ExistsCollector(); - - private BytesValues values; - - QueryCollector(ESLogger logger, Map queries, IndexSearcher searcher, IndexService percolatorIndex, List matches) { - this.logger = logger; - this.queries = queries; - this.searcher = searcher; - this.percolatorIndex = percolatorIndex; - this.matches = matches; - // TODO: when we move to a UID level mapping def on the index level, we can use that one, now, its per type, and we can't easily choose one - this.uidFieldData = percolatorIndex.fieldData().getForField(new FieldMapper.Names(UidFieldMapper.NAME), new FieldDataType("string", ImmutableSettings.builder().put("format", "paged_bytes"))); - } - - @Override - public void setScorer(Scorer scorer) throws IOException { - } - - @Override - public void collect(int doc) throws IOException { - BytesRef uid = values.getValue(doc); - if (uid == null) { - return; - } - String id = Uid.idFromUid(uid).toUtf8(); - Query query = queries.get(id); - if (query == null) { - // log??? - return; - } - // run the query - try { - collector.reset(); - searcher.search(query, collector); - if (collector.exists()) { - matches.add(id); - } - } catch (IOException e) { - logger.warn("[" + id + "] failed to execute query", e); - } - } - - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - // we use the UID because id might not be indexed - values = uidFieldData.load(context).getBytesValues(); - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return true; - } - } - - public void clearQueries() { - this.queries.clear(); - } -} diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolatorModule.java b/src/main/java/org/elasticsearch/index/percolator/PercolatorModule.java index 94e9ecedf02a9..f1bef1a7df1bf 100644 --- a/src/main/java/org/elasticsearch/index/percolator/PercolatorModule.java +++ b/src/main/java/org/elasticsearch/index/percolator/PercolatorModule.java @@ -28,7 +28,6 @@ public class PercolatorModule extends AbstractModule { @Override protected void configure() { - bind(PercolatorExecutor.class).asEagerSingleton(); bind(PercolatorService.class).asEagerSingleton(); } } diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java b/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java new file mode 100644 index 0000000000000..8bd4c64aa2230 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java @@ -0,0 +1,302 @@ +package org.elasticsearch.index.percolator; + +import org.apache.lucene.index.Term; +import org.apache.lucene.search.Query; +import org.elasticsearch.ElasticSearchException; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.HashedBytesArray; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.lucene.search.TermFilter; +import org.elasticsearch.common.lucene.search.XConstantScoreQuery; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.text.BytesText; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.cache.IndexCache; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.indexing.IndexingOperationListener; +import org.elasticsearch.index.indexing.ShardIndexingService; +import org.elasticsearch.index.mapper.DocumentTypeListener; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.internal.TypeFieldMapper; +import org.elasticsearch.index.query.IndexQueryParserService; +import org.elasticsearch.index.query.QueryParseContext; +import org.elasticsearch.index.settings.IndexSettings; +import org.elasticsearch.index.shard.AbstractIndexShardComponent; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.service.IndexShard; +import org.elasticsearch.indices.IndicesLifecycle; + +import java.util.concurrent.ConcurrentMap; + +/** + * Each shard will have a percolator registry even if there isn't a _percolator document type in the index. + * For shards with indices that have no _percolator document type, this will hold no percolate queries. + *

+ * Once a document type has been created, the real-time percolator will start to listen to write events and update the + * this registry with queries in real time. + */ +public class PercolatorQueriesRegistry extends AbstractIndexShardComponent { + + // This is a shard level service, but these below are index level service: + private final IndexQueryParserService queryParserService; + private final MapperService mapperService; + private final IndicesLifecycle indicesLifecycle; + private final IndexCache indexCache; + + private final ShardIndexingService indexingService; + + private final ConcurrentMap percolateQueries = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); + private final ShardLifecycleListener shardLifecycleListener = new ShardLifecycleListener(); + private final RealTimePercolatorOperationListener realTimePercolatorOperationListener = new RealTimePercolatorOperationListener(); + private final PercolateTypeListener percolateTypeListener = new PercolateTypeListener(); + + private volatile boolean realTimePercolatorEnabled = false; + + @Inject + public PercolatorQueriesRegistry(ShardId shardId, @IndexSettings Settings indexSettings, IndexQueryParserService queryParserService, + ShardIndexingService indexingService, IndicesLifecycle indicesLifecycle, MapperService mapperService, IndexCache indexCache) { + super(shardId, indexSettings); + this.queryParserService = queryParserService; + this.mapperService = mapperService; + this.indicesLifecycle = indicesLifecycle; + this.indexingService = indexingService; + this.indexCache = indexCache; + + indicesLifecycle.addListener(shardLifecycleListener); + mapperService.addTypeListener(percolateTypeListener); + } + + public ConcurrentMap percolateQueries() { + return percolateQueries; + } + + public void close() { + mapperService.removeTypeListener(percolateTypeListener); + indicesLifecycle.removeListener(shardLifecycleListener); + indexingService.removeListener(realTimePercolatorOperationListener); + clear(); + } + + public void clear() { + percolateQueries.clear(); + } + + void enableRealTimePercolator() { + if (!realTimePercolatorEnabled) { + indexingService.addListener(realTimePercolatorOperationListener); + realTimePercolatorEnabled = true; + } + } + + void disableRealTimePercolator() { + if (realTimePercolatorEnabled) { + indexingService.removeListener(realTimePercolatorOperationListener); + realTimePercolatorEnabled = false; + } + } + + public void addPercolateQuery(String uidAsString, BytesReference source) { + Query query = parsePercolatorDocument(uidAsString, source); + BytesText uid = new BytesText(new HashedBytesArray(Strings.toUTF8Bytes(uidAsString))); + percolateQueries.put(uid, query); + } + + public void removePercolateQuery(String uidAsString) { + BytesText uid = new BytesText(new HashedBytesArray(Strings.toUTF8Bytes(uidAsString))); + percolateQueries.remove(uid); + } + + Query parsePercolatorDocument(String uid, BytesReference source) { + String type = null; + BytesReference querySource = null; + + XContentParser parser = null; + try { + parser = XContentHelper.createParser(source); + String currentFieldName = null; + XContentParser.Token token = parser.nextToken(); // move the START_OBJECT + if (token != XContentParser.Token.START_OBJECT) { + throw new ElasticSearchException("failed to parse query [" + uid + "], not starting with OBJECT"); + } + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + if ("query".equals(currentFieldName)) { + if (type != null) { + return parseQuery(type, null, parser); + } else { + XContentBuilder builder = XContentFactory.contentBuilder(parser.contentType()); + builder.copyCurrentStructure(parser); + querySource = builder.bytes(); + builder.close(); + } + } else { + parser.skipChildren(); + } + } else if (token == XContentParser.Token.START_ARRAY) { + parser.skipChildren(); + } else if (token.isValue()) { + if ("type".equals(currentFieldName)) { + type = parser.text(); + } + } + } + return parseQuery(type, querySource, null); + } catch (Exception e) { + throw new PercolatorException(shardId().index(), "failed to parse query [" + uid + "]", e); + } finally { + if (parser != null) { + parser.close(); + } + } + } + + private Query parseQuery(String type, BytesReference querySource, XContentParser parser) { + if (type == null) { + if (parser != null) { + return queryParserService.parse(parser).query(); + } else { + return queryParserService.parse(querySource).query(); + } + } + + String[] previousTypes = QueryParseContext.setTypesWithPrevious(new String[]{type}); + try { + if (parser != null) { + return queryParserService.parse(parser).query(); + } else { + return queryParserService.parse(querySource).query(); + } + } finally { + QueryParseContext.setTypes(previousTypes); + } + } + + private class PercolateTypeListener implements DocumentTypeListener { + + @Override + public void created(String type) { + if (PercolatorService.Constants.TYPE_NAME.equals(type)) { + enableRealTimePercolator(); + } + } + + @Override + public void removed(String type) { + if (PercolatorService.Constants.TYPE_NAME.equals(type)) { + disableRealTimePercolator(); + } + } + + } + + private class ShardLifecycleListener extends IndicesLifecycle.Listener { + + @Override + public void afterIndexShardCreated(IndexShard indexShard) { + if (hasPercolatorType(indexShard)) { + enableRealTimePercolator(); + } + } + + @Override + public void afterIndexShardStarted(IndexShard indexShard) { + if (hasPercolatorType(indexShard)) { + // percolator index has started, fetch what we can from it and initialize the indices + // we have + logger.debug("loading percolator queries for index [{}] and shard[{}]...", shardId.index(), shardId.id()); + loadQueries(indexShard); + logger.trace("done loading percolator queries for index [{}] and shard[{}]", shardId.index(), shardId.id()); + } + } + + private boolean hasPercolatorType(IndexShard indexShard) { + ShardId otherShardId = indexShard.shardId(); + return shardId.equals(otherShardId) && mapperService.hasMapping(PercolatorService.Constants.TYPE_NAME); + } + + private void loadQueries(IndexShard shard) { + try { + shard.refresh(new Engine.Refresh(true)); + Engine.Searcher searcher = shard.searcher(); + try { + Query query = new XConstantScoreQuery( + indexCache.filter().cache( + new TermFilter(new Term(TypeFieldMapper.NAME, PercolatorService.Constants.TYPE_NAME)) + ) + ); + QueriesLoaderCollector queries = new QueriesLoaderCollector(PercolatorQueriesRegistry.this, logger); + searcher.searcher().search(query, queries); + percolateQueries.putAll(queries.queries()); + } finally { + searcher.release(); + } + } catch (Exception e) { + throw new PercolatorException(shardId.index(), "failed to load queries from percolator index", e); + } + } + + } + + private class RealTimePercolatorOperationListener extends IndexingOperationListener { + + @Override + public Engine.Create preCreate(Engine.Create create) { + // validate the query here, before we index + if (PercolatorService.Constants.TYPE_NAME.equals(create.type())) { + parsePercolatorDocument(create.id(), create.source()); + } + return create; + } + + @Override + public void postCreateUnderLock(Engine.Create create) { + // add the query under a doc lock + if (PercolatorService.Constants.TYPE_NAME.equals(create.type())) { + addPercolateQuery(create.id(), create.source()); + } + } + + @Override + public Engine.Index preIndex(Engine.Index index) { + // validate the query here, before we index + if (PercolatorService.Constants.TYPE_NAME.equals(index.type())) { + parsePercolatorDocument(index.id(), index.source()); + } + return index; + } + + @Override + public void postIndexUnderLock(Engine.Index index) { + // add the query under a doc lock + if (PercolatorService.Constants.TYPE_NAME.equals(index.type())) { + addPercolateQuery(index.id(), index.source()); + } + } + + @Override + public void postDeleteUnderLock(Engine.Delete delete) { + // remove the query under a lock + if (PercolatorService.Constants.TYPE_NAME.equals(delete.type())) { + removePercolateQuery(delete.id()); + } + } + + // Updating the live percolate queries for a delete by query is tricky with the current way delete by queries + // are handled. It is only possible if we put a big lock around the post delete by query hook... + + // If we implement delete by query, that just runs a query and generates delete operations in a bulk, then + // updating the live percolator is automatically supported for delete by query. +// @Override +// public void postDeleteByQuery(Engine.DeleteByQuery deleteByQuery) { +// } + } + +} diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolatorService.java b/src/main/java/org/elasticsearch/index/percolator/PercolatorService.java index 2ef270b94efbf..59515e3158d2f 100644 --- a/src/main/java/org/elasticsearch/index/percolator/PercolatorService.java +++ b/src/main/java/org/elasticsearch/index/percolator/PercolatorService.java @@ -1,307 +1,224 @@ /* - * Licensed to Elastic Search and Shay Banon under one - * or more contributor license agreements. See the NOTICE file + * Licensed to ElasticSearch and Shay Banon under one + * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information - * regarding copyright ownership. Elastic Search licenses this + * 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 + * 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. + * 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.index.percolator; -import com.google.common.collect.Maps; -import org.apache.lucene.index.AtomicReader; -import org.apache.lucene.index.AtomicReaderContext; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.Collector; +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.memory.ExtendedMemoryIndex; +import org.apache.lucene.index.memory.MemoryIndex; import org.apache.lucene.search.Filter; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; -import org.apache.lucene.search.Scorer; +import org.apache.lucene.util.CloseableThreadLocal; +import org.elasticsearch.ElasticSearchException; +import org.elasticsearch.ElasticSearchParseException; +import org.elasticsearch.action.percolate.PercolateShardRequest; +import org.elasticsearch.action.percolate.PercolateShardResponse; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.lucene.search.TermFilter; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.XConstantScoreQuery; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.AbstractIndexComponent; -import org.elasticsearch.index.Index; +import org.elasticsearch.common.text.StringText; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.fieldvisitor.UidAndSourceFieldsVisitor; -import org.elasticsearch.index.indexing.IndexingOperationListener; -import org.elasticsearch.index.mapper.internal.TypeFieldMapper; +import org.elasticsearch.index.fielddata.IndexFieldDataService; +import org.elasticsearch.index.mapper.DocumentMapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.service.IndexService; -import org.elasticsearch.index.settings.IndexSettings; -import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.service.IndexShard; -import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesService; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentMap; + +import static org.elasticsearch.index.mapper.SourceToParse.source; /** - * */ -public class PercolatorService extends AbstractIndexComponent { - - public static final String INDEX_NAME = "_percolator"; +public class PercolatorService extends AbstractComponent { + private final CloseableThreadLocal cache; private final IndicesService indicesService; - private final PercolatorExecutor percolator; - - private final ShardLifecycleListener shardLifecycleListener; - - private final RealTimePercolatorOperationListener realTimePercolatorOperationListener = new RealTimePercolatorOperationListener(); - - private final Object mutex = new Object(); - - private boolean initialQueriesFetchDone = false; - @Inject - public PercolatorService(Index index, @IndexSettings Settings indexSettings, IndicesService indicesService, - PercolatorExecutor percolator) { - super(index, indexSettings); + public PercolatorService(Settings settings, IndicesService indicesService) { + super(settings); this.indicesService = indicesService; - this.percolator = percolator; - this.shardLifecycleListener = new ShardLifecycleListener(); - this.indicesService.indicesLifecycle().addListener(shardLifecycleListener); - this.percolator.setIndicesService(indicesService); - - // if percolator is already allocated, make sure to register real time percolation - if (percolatorAllocated()) { - IndexService percolatorIndexService = percolatorIndexService(); - if (percolatorIndexService != null) { - for (IndexShard indexShard : percolatorIndexService) { - try { - indexShard.indexingService().addListener(realTimePercolatorOperationListener); - } catch (Exception e) { - // ignore - } - } + final long maxReuseBytes = settings.getAsBytesSize("indices.memory.memory_index.size_per_thread", new ByteSizeValue(1, ByteSizeUnit.MB)).bytes(); + cache = new CloseableThreadLocal() { + @Override + protected MemoryIndex initialValue() { + return new ExtendedMemoryIndex(false, maxReuseBytes); } - } + }; } - public void close() { - this.indicesService.indicesLifecycle().removeListener(shardLifecycleListener); + public PercolateShardResponse percolate(PercolateShardRequest request) { + IndexService percolateIndexService = indicesService.indexServiceSafe(request.index()); + IndexShard indexShard = percolateIndexService.shardSafe(request.shardId()); - // clean up any index that has registered real time updated from the percolator shards allocated on this node - IndexService percolatorIndexService = percolatorIndexService(); - if (percolatorIndexService != null) { - for (IndexShard indexShard : percolatorIndexService) { - try { - indexShard.indexingService().removeListener(realTimePercolatorOperationListener); - } catch (Exception e) { - // ignore - } - } + ConcurrentMap percolateQueries = indexShard.percolateRegistry().percolateQueries(); + if (percolateQueries.isEmpty()) { + return new PercolateShardResponse(StringText.EMPTY_ARRAY, request.index(), request.shardId()); } - } - public PercolatorExecutor.Response percolate(PercolatorExecutor.SourceRequest request) throws PercolatorException { - return percolator.percolate(request); - } + Tuple parseResult = parsePercolate(percolateIndexService, request.documentType(), request.documentSource()); + ParsedDocument parsedDocument = parseResult.v1(); + Query query = parseResult.v2(); - public PercolatorExecutor.Response percolate(PercolatorExecutor.DocAndSourceQueryRequest request) throws PercolatorException { - return percolator.percolate(request); - } - - private void loadQueries(String indexName) { - IndexService indexService = percolatorIndexService(); - IndexShard shard = indexService.shard(0); - shard.refresh(new Engine.Refresh(true)); - Engine.Searcher searcher = shard.searcher(); + // first, parse the source doc into a MemoryIndex + final MemoryIndex memoryIndex = cache.get(); try { - // create a query to fetch all queries that are registered under the index name (which is the type - // in the percolator). - Query query = new XConstantScoreQuery(indexQueriesFilter(indexName)); - QueriesLoaderCollector queries = new QueriesLoaderCollector(); - searcher.searcher().search(query, queries); - percolator.addQueries(queries.queries()); - } catch (IOException e) { - throw new PercolatorException(index, "failed to load queries from percolator index"); - } finally { - searcher.release(); - } - } - - private Filter indexQueriesFilter(String indexName) { - return percolatorIndexService().cache().filter().cache(new TermFilter(new Term(TypeFieldMapper.NAME, indexName))); - } - - private boolean percolatorAllocated() { - if (!indicesService.hasIndex(INDEX_NAME)) { - return false; - } - if (percolatorIndexService().numberOfShards() == 0) { - return false; - } - if (percolatorIndexService().shard(0).state() != IndexShardState.STARTED) { - return false; - } - return true; - } - - private IndexService percolatorIndexService() { - return indicesService.indexService(INDEX_NAME); - } - - class QueriesLoaderCollector extends Collector { - - private AtomicReader reader; - - private Map queries = Maps.newHashMap(); - - public Map queries() { - return this.queries; - } + // TODO: This means percolation does not support nested docs... + for (IndexableField field : parsedDocument.rootDoc().getFields()) { + if (!field.fieldType().indexed()) { + continue; + } + // no need to index the UID field + if (field.name().equals(UidFieldMapper.NAME)) { + continue; + } + TokenStream tokenStream; + try { + tokenStream = field.tokenStream(parsedDocument.analyzer()); + if (tokenStream != null) { + memoryIndex.addField(field.name(), tokenStream, field.boost()); + } + } catch (IOException e) { + throw new ElasticSearchException("Failed to create token stream", e); + } + } - @Override - public void setScorer(Scorer scorer) throws IOException { - } + final IndexSearcher searcher = memoryIndex.createSearcher(); + List matches = new ArrayList(); - @Override - public void collect(int doc) throws IOException { - // the _source is the query - UidAndSourceFieldsVisitor fieldsVisitor = new UidAndSourceFieldsVisitor(); - reader.document(doc, fieldsVisitor); - String id = fieldsVisitor.uid().id(); + IndexFieldDataService fieldDataService = percolateIndexService.fieldData(); + IndexCache indexCache = percolateIndexService.cache(); try { - final Query parseQuery = percolator.parseQuery(id, fieldsVisitor.source()); - if (parseQuery != null) { - queries.put(id, parseQuery); + if (query == null) { + Lucene.ExistsCollector collector = new Lucene.ExistsCollector(); + for (Map.Entry entry : percolateQueries.entrySet()) { + collector.reset(); + try { + searcher.search(entry.getValue(), collector); + } catch (IOException e) { + logger.warn("[" + entry.getKey() + "] failed to execute query", e); + } + + if (collector.exists()) { + matches.add(entry.getKey()); + } + } } else { - logger.warn("failed to add query [{}] - parser returned null", id); + Engine.Searcher percolatorSearcher = indexShard.searcher(); + try { + percolatorSearcher.searcher().search( + query, new QueryCollector(logger, percolateQueries, searcher, fieldDataService, matches) + ); + } catch (IOException e) { + logger.warn("failed to execute", e); + } finally { + percolatorSearcher.release(); + } } - - } catch (Exception e) { - logger.warn("failed to add query [{}]", e, id); + } finally { + // explicitly clear the reader, since we can only register on callback on SegmentReader + indexCache.clear(searcher.getIndexReader()); + fieldDataService.clear(searcher.getIndexReader()); } - } - - @Override - public void setNextReader(AtomicReaderContext context) throws IOException { - this.reader = context.reader(); - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return true; + return new PercolateShardResponse(matches.toArray(new Text[matches.size()]), request.index(), request.shardId()); + } finally { + memoryIndex.reset(); } } - class ShardLifecycleListener extends IndicesLifecycle.Listener { - - @Override - public void afterIndexShardCreated(IndexShard indexShard) { - // add a listener that will update based on changes done to the _percolate index - // the relevant indices with loaded queries - if (indexShard.shardId().index().name().equals(INDEX_NAME)) { - indexShard.indexingService().addListener(realTimePercolatorOperationListener); - } - } - - @Override - public void afterIndexShardStarted(IndexShard indexShard) { - if (indexShard.shardId().index().name().equals(INDEX_NAME)) { - // percolator index has started, fetch what we can from it and initialize the indices - // we have - synchronized (mutex) { - if (initialQueriesFetchDone) { - return; + Tuple parsePercolate(IndexService documentIndexService, String type, BytesReference docSource) throws ElasticSearchException { + Query query = null; + ParsedDocument doc = null; + XContentParser parser = null; + try { + parser = XContentFactory.xContent(docSource).createParser(docSource); + String currentFieldName = null; + XContentParser.Token token; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + // we need to check the "doc" here, so the next token will be START_OBJECT which is + // the actual document starting + if ("doc".equals(currentFieldName)) { + MapperService mapperService = documentIndexService.mapperService(); + DocumentMapper docMapper = mapperService.documentMapperWithAutoCreate(type); + doc = docMapper.parse(source(parser).type(type).flyweight(true)); } - // we load the queries for all existing indices - for (IndexService indexService : indicesService) { - // only load queries for "this" index percolator service - if (indexService.index().equals(index())) { - logger.debug("loading percolator queries for index [{}]...", indexService.index().name()); - loadQueries(indexService.index().name()); - logger.trace("done loading percolator queries for index [{}]", indexService.index().name()); + } else if (token == XContentParser.Token.START_OBJECT) { + if ("query".equals(currentFieldName)) { + if (query != null) { + throw new ElasticSearchParseException("Either specify query or filter, not both"); + } + query = documentIndexService.queryParserService().parse(parser).query(); + } else if ("filter".equals(currentFieldName)) { + if (query != null) { + throw new ElasticSearchParseException("Either specify query or filter, not both"); } + Filter filter = documentIndexService.queryParserService().parseInnerFilter(parser); + query = new XConstantScoreQuery(filter); } - initialQueriesFetchDone = true; + } else if (token == null) { + break; } } - if (!indexShard.shardId().index().equals(index())) { - // not our index, bail - return; - } - if (!percolatorAllocated()) { - return; - } - // we are only interested when the first shard on this node has been created for an index - // when it does, fetch the relevant queries if not fetched already - IndexService indexService = indicesService.indexService(indexShard.shardId().index().name()); - if (indexService == null) { - return; - } - if (indexService.numberOfShards() != 1) { - return; - } - synchronized (mutex) { - if (initialQueriesFetchDone) { - return; - } - // we load queries for this index - logger.debug("loading percolator queries for index [{}]...", indexService.index().name()); - loadQueries(index.name()); - logger.trace("done loading percolator queries for index [{}]", indexService.index().name()); - initialQueriesFetchDone = true; + } catch (IOException e) { + throw new ElasticSearchParseException("failed to parse request", e); + } finally { + if (parser != null) { + parser.close(); } } - } - class RealTimePercolatorOperationListener extends IndexingOperationListener { - - @Override - public Engine.Create preCreate(Engine.Create create) { - // validate the query here, before we index - if (create.type().equals(index().name())) { - percolator.parseQuery(create.id(), create.source()); - } - return create; + if (doc == null) { + throw new ElasticSearchParseException("No doc to percolate in the request"); } - @Override - public void postCreateUnderLock(Engine.Create create) { - // add the query under a doc lock - if (create.type().equals(index().name())) { - percolator.addQuery(create.id(), create.source()); - } - } + return new Tuple(doc, query); + } - @Override - public Engine.Index preIndex(Engine.Index index) { - // validate the query here, before we index - if (index.type().equals(index().name())) { - percolator.parseQuery(index.id(), index.source()); - } - return index; - } + public void close() { + cache.close(); + } - @Override - public void postIndexUnderLock(Engine.Index index) { - // add the query under a doc lock - if (index.type().equals(index().name())) { - percolator.addQuery(index.id(), index.source()); - } - } + public static final class Constants { + + public static final String TYPE_NAME = "_percolator"; - @Override - public void postDeleteUnderLock(Engine.Delete delete) { - // remove the query under a lock - if (delete.type().equals(index().name())) { - percolator.removeQuery(delete.id()); - } - } } + } diff --git a/src/main/java/org/elasticsearch/index/percolator/PercolatorShardModule.java b/src/main/java/org/elasticsearch/index/percolator/PercolatorShardModule.java new file mode 100644 index 0000000000000..6f260258a81b7 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/percolator/PercolatorShardModule.java @@ -0,0 +1,32 @@ +/* + * Licensed to ElasticSearch and Shay Banon 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.index.percolator; + +import org.elasticsearch.common.inject.AbstractModule; + +/** + * + */ +public class PercolatorShardModule extends AbstractModule { + + @Override + protected void configure() { + bind(PercolatorQueriesRegistry.class).asEagerSingleton(); + } +} diff --git a/src/main/java/org/elasticsearch/index/percolator/QueriesLoaderCollector.java b/src/main/java/org/elasticsearch/index/percolator/QueriesLoaderCollector.java new file mode 100644 index 0000000000000..6d4fa4101a350 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/percolator/QueriesLoaderCollector.java @@ -0,0 +1,70 @@ +package org.elasticsearch.index.percolator; + +import com.google.common.collect.Maps; +import org.apache.lucene.index.AtomicReader; +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.HashedBytesArray; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.text.BytesText; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.index.fieldvisitor.UidAndSourceFieldsVisitor; + +import java.io.IOException; +import java.util.Map; + +/** + */ +final class QueriesLoaderCollector extends Collector { + + private final Map queries = Maps.newHashMap(); + private final PercolatorQueriesRegistry percolator; + private final ESLogger logger; + + private AtomicReader reader; + + QueriesLoaderCollector(PercolatorQueriesRegistry percolator, ESLogger logger) { + this.percolator = percolator; + this.logger = logger; + } + + public Map queries() { + return this.queries; + } + + @Override + public void collect(int doc) throws IOException { + // the _source is the query + UidAndSourceFieldsVisitor fieldsVisitor = new UidAndSourceFieldsVisitor(); + reader.document(doc, fieldsVisitor); + String id = fieldsVisitor.uid().id(); + try { + final Query parseQuery = percolator.parsePercolatorDocument(id, fieldsVisitor.source()); + if (parseQuery != null) { + queries.put(new BytesText(new HashedBytesArray(Strings.toUTF8Bytes(id))), parseQuery); + } else { + logger.warn("failed to add query [{}] - parser returned null", id); + } + + } catch (Exception e) { + logger.warn("failed to add query [{}]", e, id); + } + } + + @Override + public void setNextReader(AtomicReaderContext context) throws IOException { + this.reader = context.reader(); + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } +} diff --git a/src/main/java/org/elasticsearch/index/percolator/QueryCollector.java b/src/main/java/org/elasticsearch/index/percolator/QueryCollector.java new file mode 100644 index 0000000000000..4cd4c47a56570 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/percolator/QueryCollector.java @@ -0,0 +1,87 @@ +package org.elasticsearch.index.percolator; + +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.text.BytesText; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.index.fielddata.BytesValues; +import org.elasticsearch.index.fielddata.FieldDataType; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexFieldDataService; +import org.elasticsearch.index.mapper.FieldMapper; +import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.mapper.internal.UidFieldMapper; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ConcurrentMap; + +/** + */ +final class QueryCollector extends Collector { + + private final IndexFieldData uidFieldData; + private final IndexSearcher searcher; + private final List matches; + private final ConcurrentMap queries; + private final ESLogger logger; + + private final Lucene.ExistsCollector collector = new Lucene.ExistsCollector(); + + private BytesValues values; + + QueryCollector(ESLogger logger, ConcurrentMap queries, IndexSearcher searcher, IndexFieldDataService fieldData, List matches) { + this.logger = logger; + this.queries = queries; + this.searcher = searcher; + this.matches = matches; + // TODO: when we move to a UID level mapping def on the index level, we can use that one, now, its per type, and we can't easily choose one + this.uidFieldData = fieldData.getForField(new FieldMapper.Names(UidFieldMapper.NAME), new FieldDataType("string", ImmutableSettings.builder().put("format", "paged_bytes"))); + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + } + + @Override + public void collect(int doc) throws IOException { + BytesRef uid = values.getValue(doc); + if (uid == null) { + return; + } + Text id = new BytesText(Uid.idFromUid(uid)); + Query query = queries.get(id); + if (query == null) { + // log??? + return; + } + // run the query + try { + collector.reset(); + searcher.search(query, collector); + if (collector.exists()) { + matches.add(id); + } + } catch (IOException e) { + logger.warn("[" + id + "] failed to execute query", e); + } + } + + @Override + public void setNextReader(AtomicReaderContext context) throws IOException { + // we use the UID because id might not be indexed + values = uidFieldData.load(context).getBytesValues(); + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } +} diff --git a/src/main/java/org/elasticsearch/index/service/IndexService.java b/src/main/java/org/elasticsearch/index/service/IndexService.java index bcd87881aae39..5a5884c27a54e 100644 --- a/src/main/java/org/elasticsearch/index/service/IndexService.java +++ b/src/main/java/org/elasticsearch/index/service/IndexService.java @@ -31,7 +31,6 @@ import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.gateway.IndexGateway; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.shard.service.IndexShard; @@ -53,8 +52,6 @@ public interface IndexService extends IndexComponent, Iterable { IndexSettingsService settingsService(); - PercolatorService percolateService(); - AnalysisService analysisService(); MapperService mapperService(); diff --git a/src/main/java/org/elasticsearch/index/service/InternalIndexService.java b/src/main/java/org/elasticsearch/index/service/InternalIndexService.java index fc87e9da76758..ae57c327b8675 100644 --- a/src/main/java/org/elasticsearch/index/service/InternalIndexService.java +++ b/src/main/java/org/elasticsearch/index/service/InternalIndexService.java @@ -50,7 +50,8 @@ import org.elasticsearch.index.merge.policy.MergePolicyModule; import org.elasticsearch.index.merge.policy.MergePolicyProvider; import org.elasticsearch.index.merge.scheduler.MergeSchedulerModule; -import org.elasticsearch.index.percolator.PercolatorService; +import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; +import org.elasticsearch.index.percolator.PercolatorShardModule; import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.search.stats.ShardSearchModule; import org.elasticsearch.index.settings.IndexSettings; @@ -98,8 +99,6 @@ public class InternalIndexService extends AbstractIndexComponent implements Inde private final InternalIndicesLifecycle indicesLifecycle; - private final PercolatorService percolatorService; - private final AnalysisService analysisService; private final MapperService mapperService; @@ -130,16 +129,14 @@ public class InternalIndexService extends AbstractIndexComponent implements Inde @Inject public InternalIndexService(Injector injector, Index index, @IndexSettings Settings indexSettings, NodeEnvironment nodeEnv, ThreadPool threadPool, - PercolatorService percolatorService, AnalysisService analysisService, MapperService mapperService, - IndexQueryParserService queryParserService, SimilarityService similarityService, IndexAliasesService aliasesService, - IndexCache indexCache, IndexEngine indexEngine, IndexGateway indexGateway, IndexStore indexStore, IndexSettingsService settingsService, - IndexFieldDataService indexFieldData) { + AnalysisService analysisService, MapperService mapperService, IndexQueryParserService queryParserService, + SimilarityService similarityService, IndexAliasesService aliasesService, IndexCache indexCache, IndexEngine indexEngine, + IndexGateway indexGateway, IndexStore indexStore, IndexSettingsService settingsService, IndexFieldDataService indexFieldData) { super(index, indexSettings); this.injector = injector; this.nodeEnv = nodeEnv; this.threadPool = threadPool; this.indexSettings = indexSettings; - this.percolatorService = percolatorService; this.analysisService = analysisService; this.mapperService = mapperService; this.queryParserService = queryParserService; @@ -225,11 +222,6 @@ public IndexFieldDataService fieldData() { return indexFieldData; } - @Override - public PercolatorService percolateService() { - return this.percolatorService; - } - @Override public AnalysisService analysisService() { return this.analysisService; @@ -332,6 +324,7 @@ public synchronized IndexShard createShard(int sShardId) throws ElasticSearchExc modules.add(new TranslogModule(indexSettings)); modules.add(new EngineModule(indexSettings)); modules.add(new IndexShardGatewayModule(injector.getInstance(IndexGateway.class))); + modules.add(new PercolatorShardModule()); Injector shardInjector; try { @@ -434,6 +427,13 @@ public synchronized void removeShard(int shardId, String reason) throws ElasticS logger.debug("failed to close translog", e); // ignore } + try { + // now we can close the translog + shardInjector.getInstance(PercolatorQueriesRegistry.class).close(); + } catch (Exception e) { + logger.debug("failed to close PercolatorQueriesRegistry", e); + // ignore + } // call this before we close the store, so we can release resources for it indicesLifecycle.afterIndexShardClosed(sId); diff --git a/src/main/java/org/elasticsearch/index/shard/service/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/service/IndexShard.java index f0d4a6567ebab..0f55a4549688d 100644 --- a/src/main/java/org/elasticsearch/index/shard/service/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/service/IndexShard.java @@ -39,6 +39,7 @@ import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.merge.MergeStats; +import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchService; @@ -94,6 +95,8 @@ public interface IndexShard extends IndexShardComponent { FieldDataStats fieldDataStats(String... fields); + PercolatorQueriesRegistry percolateRegistry(); + IndexShardState state(); Engine.Create prepareCreate(SourceToParse source) throws ElasticSearchException; diff --git a/src/main/java/org/elasticsearch/index/shard/service/InternalIndexShard.java b/src/main/java/org/elasticsearch/index/shard/service/InternalIndexShard.java index aa438b4cb6c2c..e4c962fc51f90 100644 --- a/src/main/java/org/elasticsearch/index/shard/service/InternalIndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/service/InternalIndexShard.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.FastByteArrayOutputStream; -import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.XFilteredQuery; import org.elasticsearch.common.metrics.MeanMetric; @@ -57,6 +56,7 @@ import org.elasticsearch.index.mapper.*; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; +import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.nested.NonNestedDocsFilter; @@ -106,6 +106,7 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I private final ShardFilterCache shardFilterCache; private final ShardIdCache shardIdCache; private final ShardFieldData shardFieldData; + private final PercolatorQueriesRegistry shardPercolator; private final Object mutex = new Object(); private final String checkIndexOnStartup; @@ -129,7 +130,8 @@ public class InternalIndexShard extends AbstractIndexShardComponent implements I @Inject public InternalIndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, Engine engine, MergeSchedulerProvider mergeScheduler, Translog translog, ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, - ShardFilterCache shardFilterCache, ShardIdCache shardIdCache, ShardFieldData shardFieldData) { + ShardFilterCache shardFilterCache, ShardIdCache shardIdCache, ShardFieldData shardFieldData, + PercolatorQueriesRegistry shardPercolator) { super(shardId, indexSettings); this.indicesLifecycle = (InternalIndicesLifecycle) indicesLifecycle; this.indexSettingsService = indexSettingsService; @@ -149,6 +151,7 @@ public InternalIndexShard(ShardId shardId, @IndexSettings Settings indexSettings this.shardFilterCache = shardFilterCache; this.shardIdCache = shardIdCache; this.shardFieldData = shardFieldData; + this.shardPercolator = shardPercolator; state = IndexShardState.CREATED; this.refreshInterval = indexSettings.getAsTime("engine.robin.refresh_interval", indexSettings.getAsTime(INDEX_REFRESH_INTERVAL, engine.defaultRefreshInterval())); @@ -482,6 +485,11 @@ public FieldDataStats fieldDataStats(String... fields) { return shardFieldData.stats(fields); } + @Override + public PercolatorQueriesRegistry percolateRegistry() { + return shardPercolator; + } + @Override public IdCacheStats idCacheStats() { return shardIdCache.stats(); diff --git a/src/main/java/org/elasticsearch/indices/InternalIndicesService.java b/src/main/java/org/elasticsearch/indices/InternalIndicesService.java index 72a494ab22bb3..3632d0f079f53 100644 --- a/src/main/java/org/elasticsearch/indices/InternalIndicesService.java +++ b/src/main/java/org/elasticsearch/indices/InternalIndicesService.java @@ -56,8 +56,6 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperServiceModule; import org.elasticsearch.index.merge.MergeStats; -import org.elasticsearch.index.percolator.PercolatorModule; -import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.index.query.IndexQueryParserModule; import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.refresh.RefreshStats; @@ -373,7 +371,6 @@ public synchronized IndexService createIndex(String sIndexName, Settings setting modules.add(new IndexAliasesServiceModule()); modules.add(new IndexGatewayModule(indexSettings, injector.getInstance(Gateway.class))); modules.add(new IndexModule(indexSettings)); - modules.add(new PercolatorModule()); Injector indexInjector; try { @@ -422,7 +419,8 @@ private void removeIndex(String index, String reason, @Nullable Executor executo ((InternalIndexService) indexService).close(reason, executor); - indexInjector.getInstance(PercolatorService.class).close(); + //TODO: close percolate queries registry +// indexInjector.getInstance(PercolatorService.class).close(); indexInjector.getInstance(IndexCache.class).close(); indexInjector.getInstance(IndexFieldDataService.class).clear(); indexInjector.getInstance(AnalysisService.class).close(); diff --git a/src/main/java/org/elasticsearch/node/internal/InternalNode.java b/src/main/java/org/elasticsearch/node/internal/InternalNode.java index 1e295cf29ea7a..b75cea30b9ef5 100644 --- a/src/main/java/org/elasticsearch/node/internal/InternalNode.java +++ b/src/main/java/org/elasticsearch/node/internal/InternalNode.java @@ -62,6 +62,8 @@ import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.http.HttpServer; import org.elasticsearch.http.HttpServerModule; +import org.elasticsearch.index.percolator.PercolatorModule; +import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.index.search.shape.ShapeModule; import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.IndicesService; @@ -162,6 +164,7 @@ public InternalNode(Settings pSettings, boolean loadConfigSettings) throws Elast modules.add(new NodeClientModule()); modules.add(new BulkUdpModule()); modules.add(new ShapeModule()); + modules.add(new PercolatorModule()); injector = modules.createInjector(); @@ -314,6 +317,8 @@ public void close() { injector.getInstance(RestController.class).close(); stopWatch.stop().start("transport"); injector.getInstance(TransportService.class).close(); + stopWatch.stop().start("percolator_service"); + injector.getInstance(PercolatorService.class).close(); for (Class plugin : pluginsService.services()) { stopWatch.stop().start("plugin(" + plugin.getName() + ")"); diff --git a/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java b/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java index 8170383042687..cd9dd2df97491 100644 --- a/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java +++ b/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.support.replication.ReplicationType; import org.elasticsearch.client.Client; @@ -117,16 +116,7 @@ public void onResponse(BulkResponse response) { } else { builder.field(Fields.OK, true); } - if (itemResponse.getResponse() instanceof IndexResponse) { - IndexResponse indexResponse = itemResponse.getResponse(); - if (indexResponse.getMatches() != null) { - builder.startArray(Fields.MATCHES); - for (String match : indexResponse.getMatches()) { - builder.value(match); - } - builder.endArray(); - } - } else if (itemResponse.getResponse() instanceof DeleteResponse) { + if (itemResponse.getResponse() instanceof DeleteResponse) { DeleteResponse deleteResponse = itemResponse.getResponse(); builder.field(Fields.FOUND, !deleteResponse.isNotFound()); } @@ -162,7 +152,6 @@ static final class Fields { static final XContentBuilderString OK = new XContentBuilderString("ok"); static final XContentBuilderString TOOK = new XContentBuilderString("took"); static final XContentBuilderString _VERSION = new XContentBuilderString("_version"); - static final XContentBuilderString MATCHES = new XContentBuilderString("matches"); static final XContentBuilderString FOUND = new XContentBuilderString("found"); } diff --git a/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java b/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java index b6f827013bcca..c00367911c46e 100644 --- a/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java +++ b/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java @@ -79,7 +79,6 @@ public void handleRequest(final RestRequest request, final RestChannel channel) indexRequest.refresh(request.paramAsBoolean("refresh", indexRequest.refresh())); indexRequest.version(RestActions.parseVersion(request)); indexRequest.versionType(VersionType.fromString(request.param("version_type"), indexRequest.versionType())); - indexRequest.percolate(request.param("percolate", null)); String sOpType = request.param("op_type"); if (sOpType != null) { if ("index".equals(sOpType)) { @@ -115,13 +114,6 @@ public void onResponse(IndexResponse response) { .field(Fields._TYPE, response.getType()) .field(Fields._ID, response.getId()) .field(Fields._VERSION, response.getVersion()); - if (response.getMatches() != null) { - builder.startArray(Fields.MATCHES); - for (String match : response.getMatches()) { - builder.value(match); - } - builder.endArray(); - } builder.endObject(); RestStatus status = OK; if (response.isCreated()) { diff --git a/src/main/java/org/elasticsearch/rest/action/percolate/RestPercolateAction.java b/src/main/java/org/elasticsearch/rest/action/percolate/RestPercolateAction.java index 8fbf4bbb80ebb..85934535d31df 100644 --- a/src/main/java/org/elasticsearch/rest/action/percolate/RestPercolateAction.java +++ b/src/main/java/org/elasticsearch/rest/action/percolate/RestPercolateAction.java @@ -20,11 +20,13 @@ package org.elasticsearch.rest.action.percolate; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.percolate.PercolateRequest; import org.elasticsearch.action.percolate.PercolateResponse; import org.elasticsearch.client.Client; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.rest.*; @@ -51,15 +53,13 @@ public RestPercolateAction(Settings settings, Client client, RestController cont @Override public void handleRequest(final RestRequest request, final RestChannel channel) { PercolateRequest percolateRequest = new PercolateRequest(request.param("index"), request.param("type")); + percolateRequest.routing(request.param("routing")); + percolateRequest.preference(request.param("preference")); percolateRequest.listenerThreaded(false); - percolateRequest.source(request.content(), request.contentUnsafe()); + percolateRequest.documentSource(request.content(), request.contentUnsafe()); // we just send a response, no need to fork percolateRequest.listenerThreaded(false); - // we don't spawn, then fork if local - percolateRequest.operationThreaded(true); - - percolateRequest.preferLocal(request.paramAsBoolean("prefer_local", percolateRequest.preferLocalShard())); client.percolate(percolateRequest, new ActionListener() { @Override public void onResponse(PercolateResponse response) { @@ -67,9 +67,27 @@ public void onResponse(PercolateResponse response) { XContentBuilder builder = RestXContentBuilder.restContentBuilder(request); builder.startObject(); - builder.field(Fields.OK, true); + builder.field(Fields.TOOK, response.getTookInMillis()); + builder.startObject(Fields._SHARDS); + builder.field(Fields.TOTAL, response.getTotalShards()); + builder.field(Fields.SUCCESSFUL, response.getSuccessfulShards()); + builder.field(Fields.FAILED, response.getFailedShards()); + if (response.getShardFailures().length > 0) { + builder.startArray(Fields.FAILURES); + for (ShardOperationFailedException shardFailure : response.getShardFailures()) { + builder.startObject(); + builder.field(Fields.INDEX, shardFailure.index()); + builder.field(Fields.SHARD, shardFailure.shardId()); + builder.field(Fields.STATUS, shardFailure.status().getStatus()); + builder.field(Fields.REASON, shardFailure.reason()); + builder.endObject(); + } + builder.endArray(); + } + builder.endObject(); + builder.startArray(Fields.MATCHES); - for (String match : response) { + for (Text match : response) { builder.value(match); } builder.endArray(); @@ -94,7 +112,16 @@ public void onFailure(Throwable e) { } static final class Fields { - static final XContentBuilderString OK = new XContentBuilderString("ok"); + static final XContentBuilderString _SHARDS = new XContentBuilderString("_shards"); + static final XContentBuilderString TOTAL = new XContentBuilderString("total"); + static final XContentBuilderString SUCCESSFUL = new XContentBuilderString("successful"); + static final XContentBuilderString FAILED = new XContentBuilderString("failed"); + static final XContentBuilderString FAILURES = new XContentBuilderString("failures"); + static final XContentBuilderString STATUS = new XContentBuilderString("status"); + static final XContentBuilderString INDEX = new XContentBuilderString("index"); + static final XContentBuilderString SHARD = new XContentBuilderString("shard"); + static final XContentBuilderString REASON = new XContentBuilderString("reason"); + static final XContentBuilderString TOOK = new XContentBuilderString("took"); static final XContentBuilderString MATCHES = new XContentBuilderString("matches"); } } diff --git a/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java b/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java index 32a524b1ad952..7bf5b84743593 100644 --- a/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java +++ b/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java @@ -69,7 +69,6 @@ public void handleRequest(final RestRequest request, final RestChannel channel) if (consistencyLevel != null) { updateRequest.consistencyLevel(WriteConsistencyLevel.fromString(consistencyLevel)); } - updateRequest.percolate(request.param("percolate", null)); updateRequest.docAsUpsert(request.paramAsBoolean("doc_as_upsert", updateRequest.docAsUpsert())); updateRequest.script(request.param("script")); updateRequest.scriptLang(request.param("lang")); @@ -144,13 +143,6 @@ public void onResponse(UpdateResponse response) { builder.endObject(); } - if (response.getMatches() != null) { - builder.startArray(Fields.MATCHES); - for (String match : response.getMatches()) { - builder.value(match); - } - builder.endArray(); - } builder.endObject(); RestStatus status = OK; if (response.isCreated()) { diff --git a/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 10bed455029f3..da947f6787dcc 100644 --- a/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -105,7 +105,7 @@ public ThreadPool(Settings settings, @Nullable NodeSettingsService nodeSettingsS .put(Names.BULK, settingsBuilder().put("type", "fixed").put("size", availableProcessors).build()) .put(Names.GET, settingsBuilder().put("type", "fixed").put("size", availableProcessors).build()) .put(Names.SEARCH, settingsBuilder().put("type", "fixed").put("size", availableProcessors * 3).put("queue_size", 1000).build()) - .put(Names.PERCOLATE, settingsBuilder().put("type", "fixed").put("size", availableProcessors).build()) + .put(Names.PERCOLATE, settingsBuilder().put("type", "fixed").put("size", availableProcessors * 3).put("queue_size", 1000).build()) .put(Names.MANAGEMENT, settingsBuilder().put("type", "scaling").put("keep_alive", "5m").put("size", 5).build()) .put(Names.FLUSH, settingsBuilder().put("type", "scaling").put("keep_alive", "5m").put("size", halfProcMaxAt5).build()) .put(Names.MERGE, settingsBuilder().put("type", "scaling").put("keep_alive", "5m").put("size", halfProcMaxAt5).build()) diff --git a/src/test/java/org/elasticsearch/benchmark/percolator/EmbeddedPercolatorBenchmarkTest.java b/src/test/java/org/elasticsearch/benchmark/percolator/EmbeddedPercolatorBenchmarkTest.java deleted file mode 100644 index 24f762515f385..0000000000000 --- a/src/test/java/org/elasticsearch/benchmark/percolator/EmbeddedPercolatorBenchmarkTest.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * Licensed to Elastic Search and Shay Banon under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Elastic Search licenses this - * file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.benchmark.percolator; - -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.common.StopWatch; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.inject.Injector; -import org.elasticsearch.common.inject.ModulesBuilder; -import org.elasticsearch.common.inject.util.Providers; -import org.elasticsearch.common.settings.ImmutableSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsModule; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexNameModule; -import org.elasticsearch.index.analysis.AnalysisModule; -import org.elasticsearch.index.cache.IndexCacheModule; -import org.elasticsearch.index.codec.CodecModule; -import org.elasticsearch.index.engine.IndexEngineModule; -import org.elasticsearch.index.mapper.MapperServiceModule; -import org.elasticsearch.index.percolator.PercolatorExecutor; -import org.elasticsearch.index.query.IndexQueryParserModule; -import org.elasticsearch.index.settings.IndexSettingsModule; -import org.elasticsearch.index.similarity.SimilarityModule; -import org.elasticsearch.indices.query.IndicesQueriesModule; -import org.elasticsearch.script.ScriptModule; -import org.elasticsearch.threadpool.ThreadPoolModule; - -import java.util.concurrent.CountDownLatch; - -import static org.elasticsearch.index.query.QueryBuilders.termQuery; - -/** - * - */ -public class EmbeddedPercolatorBenchmarkTest { - - private static long NUMBER_OF_ITERATIONS = 100000; - private static int NUMBER_OF_THREADS = 10; - private static int NUMBER_OF_QUERIES = 100; - - public static void main(String[] args) throws Exception { - Settings settings = ImmutableSettings.settingsBuilder() - .put("index.cache.filter.type", "none") - .build(); - Index index = new Index("test"); - Injector injector = new ModulesBuilder().add( - new SettingsModule(settings), - new ThreadPoolModule(settings), - new IndicesQueriesModule(), - new ScriptModule(settings), - new MapperServiceModule(), - new IndexSettingsModule(index, settings), - new IndexCacheModule(settings), - new AnalysisModule(settings), - new IndexEngineModule(settings), - new SimilarityModule(settings), - new IndexQueryParserModule(settings), - new IndexNameModule(index), - new CodecModule(settings), - new AbstractModule() { - @Override - protected void configure() { - bind(PercolatorExecutor.class).asEagerSingleton(); - bind(ClusterService.class).toProvider(Providers.of((ClusterService) null)); - } - } - ).createInjector(); - - final PercolatorExecutor percolatorExecutor = injector.getInstance(PercolatorExecutor.class); - - XContentBuilder doc = XContentFactory.jsonBuilder().startObject().startObject("doc") - .field("field1", 1) - .field("field2", "value") - .field("field3", "the quick brown fox jumped over the lazy dog") - .endObject().endObject(); - final byte[] source = doc.bytes().toBytes(); - - PercolatorExecutor.Response percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", new BytesArray(source))); - - for (int i = 0; i < NUMBER_OF_QUERIES; i++) { - percolatorExecutor.addQuery("test" + i, termQuery("field3", "quick")); - } - - - System.out.println("Warming Up (1000)"); - StopWatch stopWatch = new StopWatch().start(); - System.out.println("Running " + 1000); - for (long i = 0; i < 1000; i++) { - percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", new BytesArray(source))); - } - System.out.println("[Warmup] Percolated in " + stopWatch.stop().totalTime() + " TP Millis " + (NUMBER_OF_ITERATIONS / stopWatch.totalTime().millisFrac())); - - System.out.println("Percolating using " + NUMBER_OF_THREADS + " threads with " + NUMBER_OF_ITERATIONS + " iterations, and " + NUMBER_OF_QUERIES + " queries"); - final CountDownLatch latch = new CountDownLatch(NUMBER_OF_THREADS); - Thread[] threads = new Thread[NUMBER_OF_THREADS]; - for (int i = 0; i < threads.length; i++) { - threads[i] = new Thread(new Runnable() { - @Override - public void run() { - for (long i = 0; i < NUMBER_OF_ITERATIONS; i++) { - PercolatorExecutor.Response percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", new BytesArray(source))); - } - latch.countDown(); - } - }); - } - stopWatch = new StopWatch().start(); - for (Thread thread : threads) { - thread.start(); - } - latch.await(); - stopWatch.stop(); - System.out.println("Percolated in " + stopWatch.totalTime() + " TP Millis " + ((NUMBER_OF_ITERATIONS * NUMBER_OF_THREADS) / stopWatch.totalTime().millisFrac())); - - } -} diff --git a/src/test/java/org/elasticsearch/benchmark/percolator/SinglePercolatorStressBenchmark.java b/src/test/java/org/elasticsearch/benchmark/percolator/PercolatorStressBenchmark.java similarity index 51% rename from src/test/java/org/elasticsearch/benchmark/percolator/SinglePercolatorStressBenchmark.java rename to src/test/java/org/elasticsearch/benchmark/percolator/PercolatorStressBenchmark.java index d973f1fafde6c..f96de79acb202 100644 --- a/src/test/java/org/elasticsearch/benchmark/percolator/SinglePercolatorStressBenchmark.java +++ b/src/test/java/org/elasticsearch/benchmark/percolator/PercolatorStressBenchmark.java @@ -19,6 +19,8 @@ package org.elasticsearch.benchmark.percolator; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.percolate.PercolateResponse; import org.elasticsearch.client.Client; import org.elasticsearch.common.StopWatch; @@ -34,20 +36,21 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.query.QueryBuilders.rangeQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.node.NodeBuilder.nodeBuilder; /** * */ -public class SinglePercolatorStressBenchmark { +public class PercolatorStressBenchmark { public static void main(String[] args) throws Exception { Settings settings = settingsBuilder() .put("cluster.routing.schedule", 200, TimeUnit.MILLISECONDS) .put("gateway.type", "none") - .put(SETTING_NUMBER_OF_SHARDS, 2) - .put(SETTING_NUMBER_OF_REPLICAS, 1) + .put(SETTING_NUMBER_OF_SHARDS, 4) + .put(SETTING_NUMBER_OF_REPLICAS, 0) .build(); Node[] nodes = new Node[2]; @@ -55,34 +58,68 @@ public static void main(String[] args) throws Exception { nodes[i] = nodeBuilder().settings(settingsBuilder().put(settings).put("name", "node" + i)).node(); } - Node client = nodeBuilder().settings(settingsBuilder().put(settings).put("name", "client")).client(true).node(); + Node clientNode = nodeBuilder().settings(settingsBuilder().put(settings).put("name", "client")).client(true).node(); + Client client = clientNode.client(); - Client client1 = client.client(); - - client1.admin().indices().create(createIndexRequest("test")).actionGet(); - Thread.sleep(1000); + client.admin().indices().create(createIndexRequest("test")).actionGet(); + ClusterHealthResponse healthResponse = client.admin().cluster().prepareHealth("test") + .setWaitForGreenStatus() + .execute().actionGet(); + if (healthResponse.isTimedOut()) { + System.err.println("Quiting, because cluster health requested timed out..."); + return; + } else if (healthResponse.getStatus() != ClusterHealthStatus.GREEN) { + System.err.println("Quiting, because cluster state isn't green..."); + return; + } int COUNT = 200000; - int QUERIES = 10; + int QUERIES = 100; + int TERM_QUERIES = QUERIES / 2; + int RANGE_QUERIES = QUERIES - TERM_QUERIES; + + client.prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject().field("numeric1", 1).endObject()).execute().actionGet(); + // register queries - for (int i = 0; i < QUERIES; i++) { - client1.prepareIndex("_percolator", "test", Integer.toString(i)) + int i = 0; + for (; i < TERM_QUERIES; i++) { + client.prepareIndex("test", "_percolator", Integer.toString(i)) .setSource(jsonBuilder().startObject() .field("query", termQuery("name", "value")) .endObject()) - .setRefresh(true) .execute().actionGet(); } + int[] numbers = new int[RANGE_QUERIES]; + for (; i < QUERIES; i++) { + client.prepareIndex("test", "_percolator", Integer.toString(i)) + .setSource(jsonBuilder().startObject() + .field("query", rangeQuery("numeric1").from(i).to(i)) + .endObject()) + .execute().actionGet(); + numbers[i - TERM_QUERIES] = i; + } + StopWatch stopWatch = new StopWatch().start(); System.out.println("Percolating [" + COUNT + "] ..."); - int i = 1; - for (; i <= COUNT; i++) { - PercolateResponse percolate = client1.preparePercolate("test", "type1").setSource(source(Integer.toString(i), "value")) + for (i = 1; i <= COUNT; i++) { + XContentBuilder source; + int expectedMatches; + if (i % 2 == 0) { + source = source(Integer.toString(i), "value"); + expectedMatches = TERM_QUERIES; + } else { + int number = numbers[i % RANGE_QUERIES]; + source = source(Integer.toString(i), number); + expectedMatches = 1; + } + PercolateResponse percolate = client.preparePercolate("test", "type1") + .setSource(source) .execute().actionGet(); - if (percolate.getMatches().size() != QUERIES) { + if (percolate.getMatches().length != expectedMatches) { System.err.println("No matching number of queries"); } + if ((i % 10000) == 0) { System.out.println("Percolated " + i + " took " + stopWatch.stop().lastTaskTime()); stopWatch.start(); @@ -90,28 +127,32 @@ public static void main(String[] args) throws Exception { } System.out.println("Percolation took " + stopWatch.totalTime() + ", TPS " + (((double) COUNT) / stopWatch.totalTime().secondsFrac())); - client.close(); - + clientNode.close(); for (Node node : nodes) { node.close(); } } private static XContentBuilder source(String id, String nameValue) throws IOException { - long time = System.currentTimeMillis(); return jsonBuilder().startObject().startObject("doc") .field("id", id) - .field("numeric1", time) - .field("numeric2", time) - .field("numeric3", time) - .field("numeric4", time) - .field("numeric5", time) - .field("numeric6", time) - .field("numeric7", time) - .field("numeric8", time) - .field("numeric9", time) - .field("numeric10", time) .field("name", nameValue) .endObject().endObject(); } + + private static XContentBuilder source(String id, int number) throws IOException { + return jsonBuilder().startObject().startObject("doc") + .field("id", id) + .field("numeric1", number) + .field("numeric2", number) + .field("numeric3", number) + .field("numeric4", number) + .field("numeric5", number) + .field("numeric6", number) + .field("numeric7", number) + .field("numeric8", number) + .field("numeric9", number) + .field("numeric10", number) + .endObject().endObject(); + } } diff --git a/src/test/java/org/elasticsearch/test/integration/TestCluster.java b/src/test/java/org/elasticsearch/test/integration/TestCluster.java index f44f4bc04f7f5..7a782a55ed9ec 100644 --- a/src/test/java/org/elasticsearch/test/integration/TestCluster.java +++ b/src/test/java/org/elasticsearch/test/integration/TestCluster.java @@ -18,24 +18,11 @@ */ package org.elasticsearch.test.integration; -import static com.google.common.collect.Maps.newHashMap; -import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; -import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS; -import static org.elasticsearch.node.NodeBuilder.nodeBuilder; - -import java.io.Closeable; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Random; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicBoolean; - +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterators; +import com.google.common.collect.Sets; import org.apache.lucene.util.IOUtils; import org.elasticsearch.client.Client; import org.elasticsearch.client.transport.TransportClient; @@ -56,11 +43,15 @@ import org.elasticsearch.node.internal.InternalNode; import org.elasticsearch.transport.TransportService; -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterators; -import com.google.common.collect.Sets; +import java.io.Closeable; +import java.util.*; +import java.util.Map.Entry; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.google.common.collect.Maps.newHashMap; +import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS; +import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; +import static org.elasticsearch.node.NodeBuilder.nodeBuilder; public class TestCluster { @@ -125,6 +116,13 @@ public void ensureAtLeastNumNodes(int num) { } } + public void ensureAtLeastNumNodes(Settings settings, int num) { + int size = nodes.size(); + for (int i = size; i < num; i++) { + buildNode(settings).start(); + } + } + public void ensureAtMostNumNodes(int num) { if (nodes.size() <= num) { return; diff --git a/src/test/java/org/elasticsearch/test/integration/percolator/ConcurrentPercolatorTests.java b/src/test/java/org/elasticsearch/test/integration/percolator/ConcurrentPercolatorTests.java new file mode 100644 index 0000000000000..abbf76ad59047 --- /dev/null +++ b/src/test/java/org/elasticsearch/test/integration/percolator/ConcurrentPercolatorTests.java @@ -0,0 +1,378 @@ +/* + * Licensed to Elastic Search and Shay Banon under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Elastic Search licenses this + * file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.test.integration.percolator; + +import org.elasticsearch.action.delete.DeleteResponse; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.percolate.PercolateResponse; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.test.integration.AbstractSharedClusterTest; +import org.testng.annotations.Test; + +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.elasticsearch.index.query.QueryBuilders.boolQuery; +import static org.elasticsearch.index.query.QueryBuilders.termQuery; +import static org.elasticsearch.test.integration.percolator.SimplePercolatorTests.convertFromTextArray; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.*; + +/** + * + */ +@Test +public class ConcurrentPercolatorTests extends AbstractSharedClusterTest { + + @Test + public void testSimpleConcurrentPerculator() throws Exception { + client().admin().indices().prepareCreate("index").setSettings( + ImmutableSettings.settingsBuilder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 0) + .build() + ).execute().actionGet(); + ensureGreen(); + + final XContentBuilder onlyField1 = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field1", 1) + .endObject().endObject(); + final XContentBuilder onlyField2 = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field2", "value") + .endObject().endObject(); + final XContentBuilder bothFields = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field1", 1) + .field("field2", "value") + .endObject().endObject(); + + + // We need to index a document / define mapping, otherwise field1 doesn't get reconized as number field. + // If we don't do this, then 'test2' percolate query gets parsed as a TermQuery and not a RangeQuery. + // The percolate api doesn't parse the doc if no queries have registered, so it can't lazily create a mapping + client().prepareIndex("index", "type", "1").setSource(XContentFactory.jsonBuilder().startObject() + .field("field1", 1) + .field("field2", "value") + .endObject()).execute().actionGet(); + + client().prepareIndex("index", "_percolator", "test1") + .setSource(XContentFactory.jsonBuilder().startObject().field("query", termQuery("field2", "value")).endObject()) + .execute().actionGet(); + client().prepareIndex("index", "_percolator", "test2") + .setSource(XContentFactory.jsonBuilder().startObject().field("query", termQuery("field1", 1)).endObject()) + .execute().actionGet(); + + final CountDownLatch start = new CountDownLatch(1); + final AtomicBoolean stop = new AtomicBoolean(false); + final AtomicInteger counts = new AtomicInteger(0); + final AtomicBoolean assertionFailure = new AtomicBoolean(false); + Thread[] threads = new Thread[5]; + + for (int i = 0; i < threads.length; i++) { + Runnable r = new Runnable() { + @Override + public void run() { + try { + start.await(); + while (!stop.get()) { + int count = counts.incrementAndGet(); + if ((count > 10000)) { + stop.set(true); + } + PercolateResponse percolate; + if (count % 3 == 0) { + percolate = client().preparePercolate("index", "type").setSource(bothFields) + .execute().actionGet(); + assertThat(percolate.getMatches(), arrayWithSize(2)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContainingInAnyOrder("test1", "test2")); + } else if (count % 3 == 1) { + percolate = client().preparePercolate("index", "type").setSource(onlyField2) + .execute().actionGet(); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("test1")); + } else { + percolate = client().preparePercolate("index", "type").setSource(onlyField1) + .execute().actionGet(); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("test2")); + } + } + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (AssertionError e) { + assertionFailure.set(true); + Thread.currentThread().interrupt(); + } + } + }; + threads[i] = new Thread(r); + threads[i].start(); + } + + start.countDown(); + for (Thread thread : threads) { + thread.join(); + } + + assertThat(assertionFailure.get(), equalTo(false)); + } + + @Test + public void testConcurrentAddingAndPercolating() throws Exception { + client().admin().indices().prepareCreate("index").setSettings( + ImmutableSettings.settingsBuilder() + .put("index.number_of_shards", 2) + .put("index.number_of_replicas", 1) + .build() + ).execute().actionGet(); + ensureGreen(); + final int numIndexThreads = 3; + final int numPercolateThreads = 6; + final int numPercolatorOperationsPerThread = 1000; + + final AtomicBoolean assertionFailure = new AtomicBoolean(false); + final CountDownLatch start = new CountDownLatch(1); + final AtomicInteger runningPercolateThreads = new AtomicInteger(numPercolateThreads); + final AtomicInteger type1 = new AtomicInteger(); + final AtomicInteger type2 = new AtomicInteger(); + final AtomicInteger type3 = new AtomicInteger(); + + final AtomicInteger idGen = new AtomicInteger(); + + Thread[] indexThreads = new Thread[numIndexThreads]; + for (int i = 0; i < numIndexThreads; i++) { + Runnable r = new Runnable() { + @Override + public void run() { + try { + Random r = new Random(); + XContentBuilder onlyField1 = XContentFactory.jsonBuilder().startObject() + .field("query", termQuery("field1", "value")).endObject(); + XContentBuilder onlyField2 = XContentFactory.jsonBuilder().startObject() + .field("query", termQuery("field2", "value")).endObject(); + XContentBuilder field1And2 = XContentFactory.jsonBuilder().startObject() + .field("query", boolQuery().must(termQuery("field1", "value")).must(termQuery("field2", "value"))).endObject(); + + start.await(); + while (runningPercolateThreads.get() > 0) { + Thread.sleep(100); + int x = r.nextInt(3); + String id = Integer.toString(idGen.incrementAndGet()); + IndexResponse response; + switch (x) { + case 0: + response = client().prepareIndex("index", "_percolator", id) + .setSource(onlyField1) + .execute().actionGet(); + type1.incrementAndGet(); + break; + case 1: + response = client().prepareIndex("index", "_percolator", id) + .setSource(onlyField2) + .execute().actionGet(); + type2.incrementAndGet(); + break; + case 2: + response = client().prepareIndex("index", "_percolator", id) + .setSource(field1And2) + .execute().actionGet(); + type3.incrementAndGet(); + break; + default: + throw new IllegalStateException("Illegal x=" + x); + } + assertThat(response.getId(), equalTo(id)); + assertThat(response.getVersion(), equalTo(1l)); + } + } catch (Throwable t) { + assertionFailure.set(true); + logger.error("Error in indexing thread...", t); + } + } + }; + indexThreads[i] = new Thread(r); + indexThreads[i].start(); + } + + Thread[] percolateThreads = new Thread[numPercolateThreads]; + for (int i = 0; i < numPercolateThreads; i++) { + Runnable r = new Runnable() { + @Override + public void run() { + try { + XContentBuilder onlyField1Doc = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field1", "value") + .endObject().endObject(); + XContentBuilder onlyField2Doc = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field2", "value") + .endObject().endObject(); + XContentBuilder field1AndField2Doc = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field1", "value") + .field("field2", "value") + .endObject().endObject(); + Random random = new Random(); + start.await(); + for (int counter = 0; counter < numPercolatorOperationsPerThread; counter++) { + int x = random.nextInt(3); + int atLeastExpected; + PercolateResponse response; + switch (x) { + case 0: + atLeastExpected = type1.get(); + response = client().preparePercolate("index", "type") + .setSource(onlyField1Doc).execute().actionGet(); + assertThat(response.getShardFailures(), emptyArray()); + assertThat(response.getSuccessfulShards(), equalTo(response.getTotalShards())); + assertThat(response.getMatches().length, greaterThanOrEqualTo(atLeastExpected)); + break; + case 1: + atLeastExpected = type2.get(); + response = client().preparePercolate("index", "type") + .setSource(onlyField2Doc).execute().actionGet(); + assertThat(response.getShardFailures(), emptyArray()); + assertThat(response.getSuccessfulShards(), equalTo(response.getTotalShards())); + assertThat(response.getMatches().length, greaterThanOrEqualTo(atLeastExpected)); + break; + case 2: + atLeastExpected = type3.get(); + response = client().preparePercolate("index", "type") + .setSource(field1AndField2Doc).execute().actionGet(); + assertThat(response.getShardFailures(), emptyArray()); + assertThat(response.getSuccessfulShards(), equalTo(response.getTotalShards())); + assertThat(response.getMatches().length, greaterThanOrEqualTo(atLeastExpected)); + break; + } + } + } catch (Throwable t) { + assertionFailure.set(true); + logger.error("Error in percolate thread...", t); + } finally { + runningPercolateThreads.decrementAndGet(); + } + } + }; + percolateThreads[i] = new Thread(r); + percolateThreads[i].start(); + } + + start.countDown(); + for (Thread thread : indexThreads) { + thread.join(); + } + for (Thread thread : percolateThreads) { + thread.join(); + } + + assertThat(assertionFailure.get(), equalTo(false)); + } + + @Test + public void testConcurrentAddingAndRemovingWhilePercolating() throws Exception { + client().admin().indices().prepareCreate("index").setSettings( + ImmutableSettings.settingsBuilder() + .put("index.number_of_shards", 2) + .put("index.number_of_replicas", 1) + .build() + ).execute().actionGet(); + ensureGreen(); + final int numIndexThreads = 3; + final int numberPercolateOperation = 100; + + final AtomicBoolean assertionFailure = new AtomicBoolean(false); + final CountDownLatch start = new CountDownLatch(1); + final AtomicInteger indexOperations = new AtomicInteger(); + final AtomicInteger deleteOperations = new AtomicInteger(); + + final AtomicBoolean run = new AtomicBoolean(true); + final AtomicBoolean freeze = new AtomicBoolean(false); + final CountDownLatch[] latches = new CountDownLatch[2]; + Thread[] indexThreads = new Thread[numIndexThreads]; + for (int i = 0; i < indexThreads.length; i++) { + Runnable r = new Runnable() { + @Override + public void run() { + try { + Random r = new Random(); + XContentBuilder doc = XContentFactory.jsonBuilder().startObject() + .field("query", termQuery("field1", "value")).endObject(); + start.await(); + while (run.get()) { + if (freeze.get()) { + latches[0].countDown(); + latches[1].await(); + } + + if ((indexOperations.get() - deleteOperations.get()) > 0 && r.nextInt(100) < 19) { + String id = Integer.toString(deleteOperations.incrementAndGet()); + DeleteResponse response = client().prepareDelete("index", "_percolator", id) + .execute().actionGet(); + assertThat(response.getId(), equalTo(id)); + assertThat(response.isNotFound(), equalTo(false)); + } else { + String id = Integer.toString(indexOperations.incrementAndGet()); + IndexResponse response = client().prepareIndex("index", "_percolator", id) + .setSource(doc) + .execute().actionGet(); + assertThat(response.getId(), equalTo(id)); + } + } + } catch (Throwable t) { + run.set(false); + assertionFailure.set(true); + logger.error("Error in indexing thread...", t); + } + } + }; + indexThreads[i] = new Thread(r); + indexThreads[i].start(); + } + + XContentBuilder percolateDoc = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field1", "value") + .endObject().endObject(); + start.countDown(); + for (int counter = 0; counter < numberPercolateOperation; counter++) { + Thread.sleep(100); + latches[0] = new CountDownLatch(numIndexThreads); + latches[1] = new CountDownLatch(1); + freeze.set(true); + latches[0].await(); + + int atLeastExpected = indexOperations.get() - deleteOperations.get(); + PercolateResponse response = client().preparePercolate("index", "type") + .setSource(percolateDoc).execute().actionGet(); + assertThat(response.getShardFailures(), emptyArray()); + assertThat(response.getSuccessfulShards(), equalTo(response.getTotalShards())); + assertThat(response.getMatches().length, equalTo(atLeastExpected)); + freeze.set(false); + latches[1].countDown(); + } + run.set(false); + for (Thread thread : indexThreads) { + thread.join(); + } + assertThat(assertionFailure.get(), equalTo(false)); + } + +} diff --git a/src/test/java/org/elasticsearch/test/integration/percolator/RecoveryPercolatorTests.java b/src/test/java/org/elasticsearch/test/integration/percolator/RecoveryPercolatorTests.java index 495cf3c4c918f..8c3812e7f248e 100644 --- a/src/test/java/org/elasticsearch/test/integration/percolator/RecoveryPercolatorTests.java +++ b/src/test/java/org/elasticsearch/test/integration/percolator/RecoveryPercolatorTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.percolate.PercolateResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.Gateway; import org.elasticsearch.node.internal.InternalNode; @@ -33,10 +34,11 @@ import static org.elasticsearch.client.Requests.clusterHealthRequest; import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; -import static org.elasticsearch.index.query.QueryBuilders.termQuery; +import static org.elasticsearch.index.query.QueryBuilders.*; +import static org.elasticsearch.test.integration.percolator.SimplePercolatorTests.convertFromTextArray; +import static org.elasticsearch.test.integration.percolator.TTLPercolatorTests.ensureGreen; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.*; @Test public class RecoveryPercolatorTests extends AbstractNodesTests { @@ -68,7 +70,7 @@ public void testRestartNodePercolator1() throws Exception { client.admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); logger.info("--> register a query"); - client.prepareIndex("_percolator", "test", "kuku") + client.prepareIndex("test", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) @@ -80,7 +82,7 @@ public void testRestartNodePercolator1() throws Exception { .field("field1", "value1") .endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); client.close(); closeNode("node1"); @@ -98,7 +100,7 @@ public void testRestartNodePercolator1() throws Exception { .field("field1", "value1") .endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); } @Test @@ -114,7 +116,7 @@ public void testRestartNodePercolator2() throws Exception { client.admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); logger.info("--> register a query"); - client.prepareIndex("_percolator", "test", "kuku") + client.prepareIndex("test", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) @@ -122,13 +124,13 @@ public void testRestartNodePercolator2() throws Exception { .setRefresh(true) .execute().actionGet(); - assertThat(client.prepareCount("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(1l)); + assertThat(client.prepareCount().setTypes("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(1l)); PercolateResponse percolate = client.preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc") .field("field1", "value1") .endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); client.close(); closeNode("node1"); @@ -142,7 +144,7 @@ public void testRestartNodePercolator2() throws Exception { assertThat(clusterHealth.isTimedOut(), equalTo(false)); assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); - assertThat(client.prepareCount("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(1l)); + assertThat(client.prepareCount().setTypes("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(1l)); client.admin().indices().prepareDelete("test").execute().actionGet(); client.admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); @@ -151,16 +153,16 @@ public void testRestartNodePercolator2() throws Exception { assertThat(clusterHealth.isTimedOut(), equalTo(false)); assertThat(clusterHealth.getStatus(), equalTo(ClusterHealthStatus.YELLOW)); - assertThat(client.prepareCount("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(0l)); + assertThat(client.prepareCount().setTypes("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(0l)); percolate = client.preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc") .field("field1", "value1") .endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(0)); + assertThat(percolate.getMatches(), emptyArray()); logger.info("--> register a query"); - client.prepareIndex("_percolator", "test", "kuku") + client.prepareIndex("test", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) @@ -168,12 +170,66 @@ public void testRestartNodePercolator2() throws Exception { .setRefresh(true) .execute().actionGet(); - assertThat(client.prepareCount("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(1l)); + assertThat(client.prepareCount().setTypes("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(1l)); percolate = client.preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc") .field("field1", "value1") .endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); + } + + @Test + public void testLoadingPercolateQueriesDuringCloseAndOpen() throws Exception { + Settings settings = settingsBuilder() + .put("gateway.type", "local").build(); + logger.info("--> starting 2 nodes"); + startNode("node1", settings); + startNode("node2", settings); + + Client client = client("node1"); + client.admin().indices().prepareDelete().execute().actionGet(); + ensureGreen(client); + + client.admin().indices().prepareCreate("test") + .setSettings(settingsBuilder().put("index.number_of_shards", 2)) + .execute().actionGet(); + ensureGreen(client); + + logger.info("--> Add dummy docs"); + client.prepareIndex("test", "type1", "1").setSource("field1", 0).execute().actionGet(); + client.prepareIndex("test", "type2", "1").setSource("field1", "0").execute().actionGet(); + + logger.info("--> register a queries"); + for (int i = 1; i <= 100; i++) { + client.prepareIndex("test", "_percolator", Integer.toString(i)) + .setSource(jsonBuilder().startObject() + .field("query", rangeQuery("field1").from(0).to(i)) + // The type must be set now, because two fields with the same name exist in different types. + // Setting the type to `type1`, makes sure that the range query gets parsed to a Lucene NumericRangeQuery. + .field("type", "type1") + .endObject()) + .execute().actionGet(); + } + + logger.info("--> Percolate doc with field1=95"); + PercolateResponse response = client.preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", 95).endObject().endObject()) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(6)); + assertThat(convertFromTextArray(response.getMatches()), arrayContainingInAnyOrder("95", "96", "97", "98", "99", "100")); + + logger.info("--> Close and open index to trigger percolate queries loading..."); + client.admin().indices().prepareClose("test").execute().actionGet(); + ensureGreen(client); + client.admin().indices().prepareOpen("test").execute().actionGet(); + ensureGreen(client); + + logger.info("--> Percolate doc with field1=100"); + response = client.preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", 100).endObject().endObject()) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(1)); + assertThat(response.getMatches()[0].string(), equalTo("100")); } } diff --git a/src/test/java/org/elasticsearch/test/integration/percolator/SimplePercolatorTests.java b/src/test/java/org/elasticsearch/test/integration/percolator/SimplePercolatorTests.java index 27b88b36395dd..fae40009f645d 100644 --- a/src/test/java/org/elasticsearch/test/integration/percolator/SimplePercolatorTests.java +++ b/src/test/java/org/elasticsearch/test/integration/percolator/SimplePercolatorTests.java @@ -19,16 +19,16 @@ package org.elasticsearch.test.integration.percolator; -import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.count.CountResponse; import org.elasticsearch.action.percolate.PercolateResponse; -import org.elasticsearch.common.Priority; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.ImmutableSettings.Builder; +import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.query.FilterBuilders; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.test.integration.AbstractSharedClusterTest; import org.testng.annotations.BeforeClass; @@ -36,29 +36,201 @@ import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; -import static org.elasticsearch.index.query.QueryBuilders.termQuery; +import static org.elasticsearch.index.query.QueryBuilders.*; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.*; /** * */ public class SimplePercolatorTests extends AbstractSharedClusterTest { - + @BeforeClass public void createNodes() throws Exception { cluster().ensureAtLeastNumNodes(2); } - + + @Test + public void testSimple1() throws Exception { + client().admin().indices().prepareCreate("test").execute().actionGet(); + ensureGreen(); + + logger.info("--> Add dummy doc"); + client().prepareIndex("test", "type", "1").setSource("field", "value").execute().actionGet(); + + logger.info("--> register a queries"); + client().prepareIndex("test", "_percolator", "1") + .setSource(jsonBuilder().startObject().field("query", matchQuery("field1", "b")).field("a", "b").endObject()) + .execute().actionGet(); + client().prepareIndex("test", "_percolator", "2") + .setSource(jsonBuilder().startObject().field("query", matchQuery("field1", "c")).endObject()) + .execute().actionGet(); + client().prepareIndex("test", "_percolator", "3") + .setSource(jsonBuilder().startObject().field("query", boolQuery() + .must(matchQuery("field1", "b")) + .must(matchQuery("field1", "c")) + ).endObject()) + .execute().actionGet(); + client().prepareIndex("test", "_percolator", "4") + .setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject()) + .execute().actionGet(); + client().admin().indices().prepareRefresh("test").execute().actionGet(); + + logger.info("--> Percolate doc with field1=b"); + PercolateResponse response = client().preparePercolate("test", "type") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "b").endObject().endObject()) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(2)); + assertThat(convertFromTextArray(response.getMatches()), arrayContainingInAnyOrder("1", "4")); + + logger.info("--> Percolate doc with field1=c"); + response = client().preparePercolate("test", "type") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "c").endObject().endObject()) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(2)); + assertThat(convertFromTextArray(response.getMatches()), arrayContainingInAnyOrder("2", "4")); + + logger.info("--> Percolate doc with field1=b c"); + response = client().preparePercolate("test", "type") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "b c").endObject().endObject()) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(4)); + assertThat(convertFromTextArray(response.getMatches()), arrayContainingInAnyOrder("1", "2", "3", "4")); + + logger.info("--> Percolate doc with field1=d"); + response = client().preparePercolate("test", "type") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "d").endObject().endObject()) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(response.getMatches()), arrayContaining("4")); + + logger.info("--> Search dummy doc, percolate queries must not be included"); + SearchResponse searchResponse = client().prepareSearch("test").execute().actionGet(); + assertThat(searchResponse.getHits().totalHits(), equalTo(1L)); + assertThat(searchResponse.getHits().getAt(0).type(), equalTo("type")); + assertThat(searchResponse.getHits().getAt(0).id(), equalTo("1")); + } + + @Test + public void testSimple2() throws Exception { + client().admin().indices().prepareCreate("index").setSettings( + ImmutableSettings.settingsBuilder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 0) + .build() + ).execute().actionGet(); + client().admin().indices().prepareCreate("test").setSettings( + ImmutableSettings.settingsBuilder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 0) + .build() + ).execute().actionGet(); + ensureGreen(); + + // introduce the doc + XContentBuilder doc = XContentFactory.jsonBuilder().startObject().startObject("doc") + .field("field1", 1) + .field("field2", "value") + .endObject().endObject(); + + XContentBuilder docWithType = XContentFactory.jsonBuilder().startObject().startObject("doc").startObject("type1") + .field("field1", 1) + .field("field2", "value") + .endObject().endObject().endObject(); + + PercolateResponse response = client().preparePercolate("index", "type1").setSource(doc) + .execute().actionGet(); + assertThat(response.getMatches(), emptyArray()); + + // add first query... + client().prepareIndex("test", "_percolator", "test1") + .setSource(XContentFactory.jsonBuilder().startObject().field("query", termQuery("field2", "value")).endObject()) + .execute().actionGet(); + + response = client().preparePercolate("test", "type1").setSource(doc).execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(response.getMatches()), arrayContaining("test1")); + + response = client().preparePercolate("test", "type1").setSource(docWithType).execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(response.getMatches()), arrayContaining("test1")); + + // add second query... + client().prepareIndex("test", "_percolator", "test2") + .setSource(XContentFactory.jsonBuilder().startObject().field("query", termQuery("field1", 1)).endObject()) + .execute().actionGet(); + + response = client().preparePercolate("test", "type1") + .setSource(doc) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(2)); + assertThat(convertFromTextArray(response.getMatches()), arrayContainingInAnyOrder("test1", "test2")); + + + client().prepareDelete("test", "_percolator", "test2").execute().actionGet(); + response = client().preparePercolate("test", "type1").setSource(doc).execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(response.getMatches()), arrayContaining("test1")); + + // add a range query (cached) + // add a query + client().prepareIndex("test1", "_percolator") + .setSource( + XContentFactory.jsonBuilder().startObject().field("query", + constantScoreQuery(FilterBuilders.rangeFilter("field2").from("value").includeLower(true)) + ).endObject() + ) + .execute().actionGet(); + + response = client().preparePercolate("test", "type1").setSource(doc).execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(response.getMatches()), arrayContaining("test1")); + } + + @Test + public void testPercolateQueriesWithRouting() throws Exception { + client().admin().indices().prepareCreate("test") + .setSettings(settingsBuilder().put("index.number_of_shards", 2)) + .execute().actionGet(); + ensureGreen(); + + logger.info("--> register a queries"); + for (int i = 1; i <= 100; i++) { + client().prepareIndex("test", "_percolator", Integer.toString(i)) + .setSource(jsonBuilder().startObject().field("query", matchAllQuery()).endObject()) + .setRouting(Integer.toString(i % 2)) + .execute().actionGet(); + } + + logger.info("--> Percolate doc with no routing"); + PercolateResponse response = client().preparePercolate("test", "type") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject()) + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(100)); + + logger.info("--> Percolate doc with routing=0"); + response = client().preparePercolate("test", "type") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject()) + .setRouting("0") + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(50)); + + logger.info("--> Percolate doc with routing=1"); + response = client().preparePercolate("test", "type") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value").endObject().endObject()) + .setRouting("1") + .execute().actionGet(); + assertThat(response.getMatches(), arrayWithSize(50)); + } + @Test public void percolateOnRecreatedIndex() throws Exception { prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); - client().prepareIndex("test", "test", "1").setSource("field1", "value1").execute().actionGet(); + ensureGreen(); + client().prepareIndex("test", "test", "1").setSource("field1", "value1").execute().actionGet(); logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "kuku") + client().prepareIndex("my-queries-index", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) @@ -67,12 +239,12 @@ public void percolateOnRecreatedIndex() throws Exception { .execute().actionGet(); wipeIndex("test"); - prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); - client().prepareIndex("test", "test", "1").setSource("field1", "value1").execute().actionGet(); + ensureGreen(); + client().prepareIndex("test", "test", "1").setSource("field1", "value1").execute().actionGet(); logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "kuku") + client().prepareIndex("my-queries-index", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) @@ -100,10 +272,10 @@ public void percolateCustomAnalyzer() throws Exception { .addMapping("doc", mapping) .setSettings(builder.put("index.number_of_shards", 1)) .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); + ensureGreen(); logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "1") + client().prepareIndex("test", "_percolator", "1") .setSource(jsonBuilder().startObject() .field("source", "productizer") .field("query", QueryBuilders.constantScoreQuery(QueryBuilders.queryString("filingcategory:s"))) @@ -116,191 +288,96 @@ public void percolateCustomAnalyzer() throws Exception { .field("query", termQuery("source", "productizer")) .endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); } @Test public void registerPercolatorAndThenCreateAnIndex() throws Exception { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - try { - client().admin().indices().prepareDelete("_percolator").execute().actionGet(); - } catch (Exception e) { - // ignore - } - logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "kuku") + client().prepareIndex("my-percolate-index", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) .setRefresh(true) .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().setWaitForActiveShards(2).execute().actionGet(); client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); + ensureGreen(); - PercolateResponse percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc") - .field("field1", "value1") - .endObject().endObject()) + PercolateResponse percolate = client().preparePercolate("my-percolate-index", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value1").endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); - percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject() - .startObject("doc").field("field1", "value1").endObject() - .field("query", matchAllQuery()) - .endObject()) + percolate = client().preparePercolate("my-percolate-index", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value1").endObject().field("query", matchAllQuery()).endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); } @Test public void createIndexAndThenRegisterPercolator() throws Exception { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - try { - client().admin().indices().prepareDelete("_percolator").execute().actionGet(); - } catch (Exception e) { - // ignore - } - client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); + ensureGreen(); logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "kuku") + client().prepareIndex("test", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().setWaitForActiveShards(4).execute().actionGet(); + + refresh(); + CountResponse countResponse = client().prepareCount() + .setQuery(matchAllQuery()).setTypes("_percolator") + .execute().actionGet(); + assertThat(countResponse.getCount(), equalTo(1l)); + for (int i = 0; i < 10; i++) { - PercolateResponse percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc") - .field("field1", "value1") - .endObject().endObject()) + PercolateResponse percolate = client().preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value1").endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); } for (int i = 0; i < 10; i++) { - PercolateResponse percolate = client().preparePercolate("test", "type1").setPreferLocal(false).setSource(jsonBuilder().startObject().startObject("doc") - .field("field1", "value1") - .endObject().endObject()) + PercolateResponse percolate = client().preparePercolate("test", "type1") + .setPreference("_local") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value1").endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); + assertThat(percolate.getMatches(), arrayWithSize(1)); } - logger.info("--> delete the index"); - - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - logger.info("--> make sure percoalted queries for it have been deleted as well"); - assertThat(client().prepareCount("_percolator").setQuery(matchAllQuery()).execute().actionGet().getCount(), equalTo(0l)); - } - - @Test - public void percolateOnIndexOperation() throws Exception { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - try { - client().admin().indices().prepareDelete("_percolator").execute().actionGet(); - } catch (Exception e) { - // ignore - } - - client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 2)).execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); - - logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "kuku") - .setSource(jsonBuilder().startObject() - .field("color", "blue") - .field("query", termQuery("field1", "value1")) - .endObject()) - .setRefresh(true) + logger.info("--> delete the index"); + client().admin().indices().prepareDelete("test").execute().actionGet(); + logger.info("--> make sure percolated queries for it have been deleted as well"); + countResponse = client().prepareCount() + .setQuery(matchAllQuery()).setTypes("_percolator") .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().setWaitForActiveShards(4).execute().actionGet(); - - for (int i = 0; i < 10; i++) { - IndexResponse index = client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field1", "value1") - .setPercolate("*").execute().actionGet(); - assertThat(index.getMatches().size(), equalTo(1)); - assertThat(index.getMatches(), hasItem("kuku")); - } - - for (int i = 0; i < 10; i++) { - IndexResponse index = client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field1", "value1") - .setPercolate("color:blue").execute().actionGet(); - assertThat(index.getMatches().size(), equalTo(1)); - assertThat(index.getMatches(), hasItem("kuku")); - } - - for (int i = 0; i < 10; i++) { - IndexResponse index = client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field1", "value1") - .setPercolate("color:green").execute().actionGet(); - assertThat(index.getMatches().size(), equalTo(0)); - } - - // test bulk - BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); - for (int i = 0; i < 10; i++) { - bulkRequestBuilder.add(client().prepareIndex("test", "type1", Integer.toString(i)).setSource("field1", "value1") - .setPercolate("*")); - } - BulkResponse bulkResponse = bulkRequestBuilder.execute().actionGet(); - assertThat(bulkResponse.hasFailures(), equalTo(false)); - for (BulkItemResponse bulkItemResponse : bulkResponse) { - IndexResponse index = bulkItemResponse.getResponse(); - assertThat(index.getMatches().size(), equalTo(1)); - assertThat(index.getMatches(), hasItem("kuku")); - } + assertThat(countResponse.getCount(), equalTo(0l)); } @Test public void multiplePercolators() throws Exception { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - try { - client().admin().indices().prepareDelete("_percolator").execute().actionGet(); - } catch (Exception e) { - // ignore - } client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); + ensureGreen(); logger.info("--> register a query 1"); - client().prepareIndex("_percolator", "test", "kuku") + client().prepareIndex("test", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) .setRefresh(true) .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().setWaitForActiveShards(4).execute().actionGet(); logger.info("--> register a query 2"); - client().prepareIndex("_percolator", "test", "bubu") + client().prepareIndex("test", "_percolator", "bubu") .setSource(jsonBuilder().startObject() .field("color", "green") .field("query", termQuery("field1", "value2")) @@ -308,58 +385,42 @@ public void multiplePercolators() throws Exception { .setRefresh(true) .execute().actionGet(); - - PercolateResponse percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc") - .field("field1", "value1") - .endObject().endObject()) + PercolateResponse percolate = client().preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value1").endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); - assertThat(percolate.getMatches(), hasItem("kuku")); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("kuku")); - percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc").startObject("type1") - .field("field1", "value2") - .endObject().endObject().endObject()) + percolate = client().preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").startObject("type1").field("field1", "value2").endObject().endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); - assertThat(percolate.getMatches(), hasItem("bubu")); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("bubu")); } @Test public void dynamicAddingRemovingQueries() throws Exception { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - try { - client().admin().indices().prepareDelete("_percolator").execute().actionGet(); - } catch (Exception e) { - // ignore - } - client().admin().indices().prepareCreate("test").setSettings(settingsBuilder().put("index.number_of_shards", 1)).execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); + ensureGreen(); logger.info("--> register a query 1"); - client().prepareIndex("_percolator", "test", "kuku") + client().prepareIndex("test", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) .setRefresh(true) .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().setWaitForActiveShards(4).execute().actionGet(); - PercolateResponse percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc") - .field("field1", "value1") - .endObject().endObject()) + PercolateResponse percolate = client().preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").field("field1", "value1").endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); - assertThat(percolate.getMatches(), hasItem("kuku")); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("kuku")); logger.info("--> register a query 2"); - client().prepareIndex("_percolator", "test", "bubu") + client().prepareIndex("test", "_percolator", "bubu") .setSource(jsonBuilder().startObject() .field("color", "green") .field("query", termQuery("field1", "value2")) @@ -367,15 +428,14 @@ public void dynamicAddingRemovingQueries() throws Exception { .setRefresh(true) .execute().actionGet(); - percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc").startObject("type1") - .field("field1", "value2") - .endObject().endObject().endObject()) + percolate = client().preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").startObject("type1").field("field1", "value2").endObject().endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); - assertThat(percolate.getMatches(), hasItem("bubu")); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("bubu")); logger.info("--> register a query 3"); - client().prepareIndex("_percolator", "test", "susu") + client().prepareIndex("test", "_percolator", "susu") .setSource(jsonBuilder().startObject() .field("color", "red") .field("query", termQuery("field1", "value2")) @@ -383,40 +443,25 @@ public void dynamicAddingRemovingQueries() throws Exception { .setRefresh(true) .execute().actionGet(); - percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject() - .startObject("doc").startObject("type1") - .field("field1", "value2") - .endObject().endObject() - - .field("query", termQuery("color", "red")) - - .endObject()) + percolate = client().preparePercolate("test", "type1") + .setSource(jsonBuilder().startObject().startObject("doc").startObject("type1").field("field1", "value2").endObject().endObject() + .field("query", termQuery("color", "red")).endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); - assertThat(percolate.getMatches(), hasItem("susu")); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("susu")); logger.info("--> deleting query 1"); - client().prepareDelete("_percolator", "test", "kuku").setRefresh(true).execute().actionGet(); + client().prepareDelete("test", "_percolator", "kuku").setRefresh(true).execute().actionGet(); percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject().startObject("doc").startObject("type1") .field("field1", "value1") .endObject().endObject().endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(0)); + assertThat(percolate.getMatches(), emptyArray()); } @Test public void percolateWithSizeField() throws Exception { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore - } - try { - client().admin().indices().prepareDelete("_percolator").execute().actionGet(); - } catch (Exception e) { - // ignore - } String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1") .startObject("_size").field("enabled", true).field("stored", "yes").endObject() .endObject().endObject().string(); @@ -425,16 +470,15 @@ public void percolateWithSizeField() throws Exception { .setSettings(settingsBuilder().put("index.number_of_shards", 2)) .addMapping("type1", mapping) .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); + ensureGreen(); logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "kuku") + client().prepareIndex("test", "_percolator", "kuku") .setSource(jsonBuilder().startObject() .field("query", termQuery("field1", "value1")) .endObject()) .setRefresh(true) .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().setWaitForActiveShards(4).execute().actionGet(); logger.info("--> percolate a document"); PercolateResponse percolate = client().preparePercolate("test", "type1").setSource(jsonBuilder().startObject() @@ -443,52 +487,19 @@ public void percolateWithSizeField() throws Exception { .endObject().endObject() .endObject()) .execute().actionGet(); - assertThat(percolate.getMatches().size(), equalTo(1)); - assertThat(percolate.getMatches(), hasItem("kuku")); + assertThat(percolate.getMatches(), arrayWithSize(1)); + assertThat(convertFromTextArray(percolate.getMatches()), arrayContaining("kuku")); } - @Test - public void testThatPercolatingWithTimeToLiveWorks() throws Exception { - try { - client().admin().indices().prepareDelete("test").execute().actionGet(); - } catch (Exception e) { - // ignore + public static String[] convertFromTextArray(Text[] texts) { + if (texts.length == 0) { + return Strings.EMPTY_ARRAY; } - try { - client().admin().indices().prepareDelete("_percolator").execute().actionGet(); - } catch (Exception e) { - // ignore + String[] strings = new String[texts.length]; + for (int i = 0; i < texts.length; i++) { + strings[i] = texts[i].string(); } - - String mapping = XContentFactory.jsonBuilder().startObject().startObject("type1") - .startObject("_ttl").field("enabled", true).field("default", "60d").endObject() - .startObject("_timestamp").field("enabled", true).endObject() - .endObject().endObject().string(); - - client().admin().indices().prepareCreate("test") - .setSettings(settingsBuilder().put("index.number_of_shards", 2)) - .addMapping("type1", mapping) - .execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); - - client().prepareIndex("_percolator", "test", "kuku").setSource(jsonBuilder() - .startObject() - .startObject("query") - .startObject("term") - .field("field1", "value1") - .endObject() - .endObject() - .endObject() - ).setRefresh(true).execute().actionGet(); - client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().setWaitForActiveShards(4).execute().actionGet(); - - PercolateResponse percolateResponse = client().preparePercolate("test", "type1").setSource(jsonBuilder() - .startObject() - .startObject("doc") - .field("field1", "value1") - .endObject() - .endObject() - ).execute().actionGet(); - assertThat(percolateResponse.getMatches(), hasItem("kuku")); + return strings; } + } diff --git a/src/test/java/org/elasticsearch/test/integration/percolator/TTLPercolatorTests.java b/src/test/java/org/elasticsearch/test/integration/percolator/TTLPercolatorTests.java new file mode 100644 index 0000000000000..4c474378fa32e --- /dev/null +++ b/src/test/java/org/elasticsearch/test/integration/percolator/TTLPercolatorTests.java @@ -0,0 +1,99 @@ +package org.elasticsearch.test.integration.percolator; + +import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus; +import org.elasticsearch.action.percolate.PercolateResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.Requests; +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.test.integration.AbstractNodesTests; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.Test; + +import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.test.integration.percolator.SimplePercolatorTests.convertFromTextArray; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.*; + +/** + */ +public class TTLPercolatorTests extends AbstractNodesTests { + + @Test + public void testThatPercolatingWithTimeToLiveWorks() throws Exception { + long purgeInterval = 200; + Settings settings = settingsBuilder() + .put("gateway.type", "none") + .put("indices.ttl.interval", purgeInterval).build(); // <-- For testing ttl. + logger.info("--> starting 2 nodes"); + startNode("node1", settings); + startNode("node2", settings); + + Client client = client("node1"); + client.admin().indices().prepareDelete().execute().actionGet(); + ensureGreen(client); + + String mapping = XContentFactory.jsonBuilder().startObject().startObject("_percolator") + .startObject("_ttl").field("enabled", true).endObject() + .startObject("_timestamp").field("enabled", true).endObject() + .endObject().endObject().string(); + + client.admin().indices().prepareCreate("test") + .setSettings(settingsBuilder().put("index.number_of_shards", 2)) + .addMapping("_percolator", mapping) + .addMapping("type1", mapping) + .execute().actionGet(); + ensureGreen(client); + + long ttl = 1500; + long now = System.currentTimeMillis(); + client.prepareIndex("test", "_percolator", "kuku").setSource(jsonBuilder() + .startObject() + .startObject("query") + .startObject("term") + .field("field1", "value1") + .endObject() + .endObject() + .endObject() + ).setRefresh(true).setTTL(ttl).execute().actionGet(); + + PercolateResponse percolateResponse = client.preparePercolate("test", "type1").setSource(jsonBuilder() + .startObject() + .startObject("doc") + .field("field1", "value1") + .endObject() + .endObject() + ).execute().actionGet(); + assertThat(convertFromTextArray(percolateResponse.getMatches()), arrayContaining("kuku")); + + long now1 = System.currentTimeMillis(); + if ((now1 - now) <= (ttl + purgeInterval)) { + logger.info("Waiting for ttl purger..."); + Thread.sleep((ttl + purgeInterval) - (now1 - now)); + } + percolateResponse = client.preparePercolate("test", "type1").setSource(jsonBuilder() + .startObject() + .startObject("doc") + .field("field1", "value1") + .endObject() + .endObject() + ).execute().actionGet(); + assertThat(percolateResponse.getMatches(), emptyArray()); + } + + @AfterMethod + public void cleanAndCloseNodes() throws Exception { + closeAllNodes(); + } + + public static void ensureGreen(Client client) { + ClusterHealthResponse actionGet = client.admin().cluster() + .health(Requests.clusterHealthRequest().waitForGreenStatus().waitForEvents(Priority.LANGUID).waitForRelocatingShards(0)).actionGet(); + assertThat(actionGet.isTimedOut(), equalTo(false)); + assertThat(actionGet.getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + +} diff --git a/src/test/java/org/elasticsearch/test/integration/update/UpdateTests.java b/src/test/java/org/elasticsearch/test/integration/update/UpdateTests.java index 0f4aa629883a4..5094c74617e26 100644 --- a/src/test/java/org/elasticsearch/test/integration/update/UpdateTests.java +++ b/src/test/java/org/elasticsearch/test/integration/update/UpdateTests.java @@ -37,7 +37,6 @@ import java.util.concurrent.CountDownLatch; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.*; @@ -334,19 +333,6 @@ public void testUpdate() throws Exception { assertThat(getResponse.isExists(), equalTo(false)); } - // check percolation - client().prepareIndex("test", "type1", "1").setSource("field", 1).execute().actionGet(); - logger.info("--> register a query"); - client().prepareIndex("_percolator", "test", "1") - .setSource(jsonBuilder().startObject() - .field("query", termQuery("field", 2)) - .endObject()) - .setRefresh(true) - .execute().actionGet(); - ensureGreen(); - updateResponse = client().prepareUpdate("test", "type1", "1").setScript("ctx._source.field += 1").setPercolate("*").execute().actionGet(); - assertThat(updateResponse.getMatches().size(), equalTo(1)); - // check TTL is kept after an update without TTL client().prepareIndex("test", "type1", "2").setSource("field", 1).setTTL(86400000L).setRefresh(true).execute().actionGet(); GetResponse getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet(); diff --git a/src/test/java/org/elasticsearch/test/unit/index/cache/id/SimpleIdCacheTests.java b/src/test/java/org/elasticsearch/test/unit/index/cache/id/SimpleIdCacheTests.java index 82215f185306c..49eaf48cd8ada 100644 --- a/src/test/java/org/elasticsearch/test/unit/index/cache/id/SimpleIdCacheTests.java +++ b/src/test/java/org/elasticsearch/test/unit/index/cache/id/SimpleIdCacheTests.java @@ -49,7 +49,6 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.internal.ParentFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; -import org.elasticsearch.index.percolator.PercolatorService; import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.service.IndexService; import org.elasticsearch.index.settings.IndexSettingsService; @@ -312,11 +311,6 @@ public IndexSettingsService settingsService() { return null; } - @Override - public PercolatorService percolateService() { - return null; - } - @Override public AnalysisService analysisService() { return null; diff --git a/src/test/java/org/elasticsearch/test/unit/index/percolator/PercolatorExecutorTests.java b/src/test/java/org/elasticsearch/test/unit/index/percolator/PercolatorExecutorTests.java deleted file mode 100644 index c9de5e6092ab4..0000000000000 --- a/src/test/java/org/elasticsearch/test/unit/index/percolator/PercolatorExecutorTests.java +++ /dev/null @@ -1,252 +0,0 @@ -/* - * Licensed to Elastic Search and Shay Banon under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Elastic Search licenses this - * file to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.test.unit.index.percolator; - -import java.io.IOException; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import org.elasticsearch.cache.recycler.CacheRecyclerModule; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.inject.Injector; -import org.elasticsearch.common.inject.ModulesBuilder; -import org.elasticsearch.common.inject.util.Providers; -import org.elasticsearch.common.settings.ImmutableSettings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsModule; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexNameModule; -import org.elasticsearch.index.analysis.AnalysisModule; -import org.elasticsearch.index.cache.IndexCacheModule; -import org.elasticsearch.index.codec.CodecModule; -import org.elasticsearch.index.engine.IndexEngineModule; -import org.elasticsearch.index.mapper.MapperServiceModule; -import org.elasticsearch.index.percolator.PercolatorExecutor; -import org.elasticsearch.index.query.FilterBuilders; -import org.elasticsearch.index.query.IndexQueryParserModule; -import org.elasticsearch.index.settings.IndexSettingsModule; -import org.elasticsearch.index.settings.IndexSettingsService; -import org.elasticsearch.index.similarity.SimilarityModule; -import org.elasticsearch.indices.query.IndicesQueriesModule; -import org.elasticsearch.script.ScriptModule; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.threadpool.ThreadPoolModule; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeTest; -import org.testng.annotations.Test; - -import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery; -import static org.elasticsearch.index.query.QueryBuilders.termQuery; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.*; - -/** - * - */ -@Test -public class PercolatorExecutorTests { - - private Injector injector; - - private PercolatorExecutor percolatorExecutor; - - @BeforeTest - public void buildPercolatorService() { - Settings settings = ImmutableSettings.settingsBuilder() - //.put("index.cache.filter.type", "none") - .build(); - Index index = new Index("test"); - injector = new ModulesBuilder().add( - new CacheRecyclerModule(settings), - new IndexSettingsModule(index, settings), - new CodecModule(settings), - new SettingsModule(settings), - new ThreadPoolModule(settings), - new ScriptModule(settings), - new IndicesQueriesModule(), - new MapperServiceModule(), - new IndexCacheModule(settings), - new AnalysisModule(settings), - new IndexEngineModule(settings), - new SimilarityModule(settings), - new IndexQueryParserModule(settings), - new IndexNameModule(index), - new AbstractModule() { - @Override - protected void configure() { - bind(PercolatorExecutor.class).asEagerSingleton(); - bind(ClusterService.class).toProvider(Providers.of((ClusterService) null)); - } - } - ).createInjector(); - - percolatorExecutor = injector.getInstance(PercolatorExecutor.class); - } - - @AfterClass - public void close() { - injector.getInstance(ThreadPool.class).shutdownNow(); - } - - @Test - public void testSimplePercolator() throws Exception { - // introduce the doc - XContentBuilder doc = XContentFactory.jsonBuilder().startObject().startObject("doc") - .field("field1", 1) - .field("field2", "value") - .endObject().endObject(); - BytesReference source = doc.bytes(); - - XContentBuilder docWithType = XContentFactory.jsonBuilder().startObject().startObject("doc").startObject("type1") - .field("field1", 1) - .field("field2", "value") - .endObject().endObject().endObject(); - BytesReference sourceWithType = docWithType.bytes(); - - percolatorExecutor.clearQueries(); // remove all previously added queries - PercolatorExecutor.Response percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", source)); - assertThat(percolate.matches(), hasSize(0)); - - // add a query - percolatorExecutor.addQuery("test1", termQuery("field2", "value")); - - percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", source)); - assertThat(percolate.matches(), hasSize(1)); - assertThat(percolate.matches(), hasItem("test1")); - - percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", sourceWithType)); - assertThat(percolate.matches(), hasSize(1)); - assertThat(percolate.matches(), hasItem("test1")); - - percolatorExecutor.addQuery("test2", termQuery("field1", 1)); - - percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", source)); - assertThat(percolate.matches(), hasSize(2)); - assertThat(percolate.matches(), hasItems("test1", "test2")); - - - percolatorExecutor.removeQuery("test2"); - percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", source)); - assertThat(percolate.matches(), hasSize(1)); - assertThat(percolate.matches(), hasItems("test1")); - - // add a range query (cached) - // add a query - percolatorExecutor.addQuery("test1", constantScoreQuery(FilterBuilders.rangeFilter("field2").from("value").includeLower(true))); - - percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", source)); - assertThat(percolate.matches(), hasSize(1)); - assertThat(percolate.matches(), hasItem("test1")); - } - - @Test - public void testConcurrentPerculator() throws InterruptedException, IOException { - // introduce the doc - XContentBuilder bothQueriesB = XContentFactory.jsonBuilder().startObject().startObject("doc") - .field("field1", 1) - .field("field2", "value") - .endObject().endObject(); - final BytesReference bothQueries = bothQueriesB.bytes(); - - XContentBuilder onlyTest1B = XContentFactory.jsonBuilder().startObject().startObject("doc") - .field("field2", "value") - .endObject().endObject(); - XContentBuilder onlyTest2B = XContentFactory.jsonBuilder().startObject().startObject("doc") - .field("field1", 1) - .endObject().endObject(); - final BytesReference onlyTest1 = onlyTest1B.bytes(); - final BytesReference onlyTest2 = onlyTest2B.bytes(); - final PercolatorExecutor executor = this.percolatorExecutor; - percolatorExecutor.clearQueries(); // remove all previously added queries - // this adds the mapping and ensures that we do a NRQ for field 1 - PercolatorExecutor.Response percolate = percolatorExecutor.percolate(new PercolatorExecutor.SourceRequest("type1", bothQueries)); - assertThat(percolate.matches(), hasSize(0)); - executor.addQuery("test1", termQuery("field2", "value")); - executor.addQuery("test2", termQuery("field1", 1)); - - final IndexSettingsService settingsService = injector.getInstance(IndexSettingsService.class); - final CountDownLatch start = new CountDownLatch(1); - final AtomicBoolean stop = new AtomicBoolean(false); - final AtomicInteger counts = new AtomicInteger(0); - Thread[] threads = new Thread[5]; - - for (int i = 0; i < threads.length; i++) { - threads[i] = new Thread() { - public void run() { - try { - - - start.await(); - - PercolatorExecutor.Response percolate; - while(!stop.get()) { - int count = counts.incrementAndGet(); - if ((count % 100) == 0) { - ImmutableSettings.Builder builder = ImmutableSettings.settingsBuilder(); - builder.put(PercolatorExecutor.PERCOLATE_POOL_MAX_MEMORY, 1 + (counts.get() % 10), ByteSizeUnit.MB); - builder.put(PercolatorExecutor.PERCOLATE_POOL_SIZE, 1 + (counts.get() % 10)); - builder.put(PercolatorExecutor.PERCOLATE_TIMEOUT, 1 + (counts.get() % 1000), TimeUnit.MILLISECONDS); - settingsService.refreshSettings(builder.build()); - } - - if ((count > 10000)) { - stop.set(true); - } - if (count % 3 == 0) { - percolate = executor.percolate(new PercolatorExecutor.SourceRequest("type1", bothQueries)); - assertThat(percolate.matches(), hasSize(2)); - assertThat(percolate.matches(), hasItems("test1", "test2")); - } else if (count % 3 == 1) { - percolate = executor.percolate(new PercolatorExecutor.SourceRequest("type1", onlyTest1)); - assertThat(percolate.matches(), hasSize(1)); - assertThat(percolate.matches(), hasItems("test1")); - } else { - percolate = executor.percolate(new PercolatorExecutor.SourceRequest("type1", onlyTest2)); - assertThat(percolate.matches(), hasSize(1)); - assertThat(percolate.matches(), hasItems("test2")); - } - - - } - - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return; - } - } - }; - threads[i].start(); - } - - start.countDown(); - for (Thread thread : threads) { - thread.join(); - } - - } -}