diff --git a/docs/reference/query-dsl/filters/terms-filter.asciidoc b/docs/reference/query-dsl/filters/terms-filter.asciidoc index fa36fc0da00f9..a5811a1e759dd 100644 --- a/docs/reference/query-dsl/filters/terms-filter.asciidoc +++ b/docs/reference/query-dsl/filters/terms-filter.asciidoc @@ -30,7 +30,7 @@ them in the more compact model that terms filter provides. The `execution` option now has the following options : [horizontal] -`plain`:: +`plain`:: The default. Works as today. Iterates over all the terms, building a bit set matching it, and filtering. The total filter is cached. @@ -38,22 +38,22 @@ The `execution` option now has the following options : `fielddata`:: Generates a terms filters that uses the fielddata cache to compare terms. This execution mode is great to use when filtering - on a field that is already loaded into the fielddata cache from + on a field that is already loaded into the fielddata cache from faceting, sorting, or index warmers. When filtering on a large number of terms, this execution can be considerably faster than the other modes. The total filter is not cached unless explicitly configured to do so. -`bool`:: +`bool`:: Generates a term filter (which is cached) for each term, and wraps those in a bool filter. The bool filter itself is not cached as it can operate very quickly on the cached term filters. -`and`:: +`and`:: Generates a term filter (which is cached) for each term, and wraps those in an and filter. The and filter itself is not cached. -`or`:: +`or`:: Generates a term filter (which is cached) for each term, and wraps those in an or filter. The or filter itself is not cached. Generally, the `bool` execution mode should be preferred. @@ -102,25 +102,25 @@ lookup mechanism. The terms lookup mechanism supports the following options: [horizontal] -`index`:: +`index`:: The index to fetch the term values from. Defaults to the current index. -`type`:: +`type`:: The type to fetch the term values from. -`id`:: +`id`:: The id of the document to fetch the term values from. -`path`:: +`path`:: The field specified as path to fetch the actual values for the `terms` filter. -`routing`:: +`routing`:: A custom routing value to be used when retrieving the external terms doc. -`cache`:: +`cache`:: Whether to cache the filter built from the retrieved document (`true` - default) or whether to fetch and rebuild the filter on every request (`false`). See "<>" below @@ -136,6 +136,112 @@ across all nodes if the "reference" terms data is not large. The lookup terms filter will prefer to execute the get request on a local node if possible, reducing the need for networking. +[float] +==== Terms lookup by query mechanism + +added[1.2.0] + +.Experimental! +[IMPORTANT] +===== +This feature is marked as experimental, and may be subject to change in the +future. If you use this feature, please let us know your experience with it! +===== + +The terms filter by query feature allows the lookup of terms from +documents matching a query/filter. This functionality is similar to +the "<>" and "<>" +functionality without the limiations. The lookup query can be executed +over multiple indices, shards, and types. + +The terms lookup by query mechanism supports the following options: + +[horizontal] +`indices` or `index`:: + One or more indices to execute the lookup query against. Default's + to all indices if not specified. + +`types` or `type`:: + One or more types to execute against. Default's to all types of + the configured indices. + +`path`:: + The field to fetch the actual values from the documents matching + the lookup query. + +`filter`:: + The query filter documents must match for their terms to be collected + as part of the query lookup. + +`max_terms_per_shard`:: + The maximum number of terms to collect from each shard. Default's to + all terms. + +`routing`:: + A custom routing value to be used when executing the lookup query. + +`cache`:: + Whether to cache the filter built from the retrieved terms + (`true` - default) or whether to fetch and rebuild the filter on every + request (`false`). See "<>" + below. + +`bloom_filter`:: + Configures the query lookup to gather terms within a bloom filter for + more compact term representation at the cost of lookup precision. See + the "<>" section below. + +The values for the `terms` filter will be fetched from the `path` field of +documents matching the lookup filter. If `max_terms_per_shard` is set, then +the number of terms gathered will be at most `max_terms_per_shard` * NUM_SHARDS. +If the source field is a text based field, the cost of gathering a large number +of terms can be quite expensive due to network latency. In this situation it is +best to use numeric fields or configure the lookup to use a +"<>". + +The terms lookup currently only uses the `fielddata` exection mode so proper "<>" is +recommended. Since the filter uses the fielddata cache by default, the resulting filter is not cached unless +configured to do so. + +["float",id="query-dsl-terms-filter-lookup-bloom"] +==== Bloom filter support + +When performing a lookup query against a set of text based terms with a high +cardinality the cost of transfering these terms over the network can be +quite expensive resulting in slow response times. If precision is not +critical, performance can be considerably better by using a +http://en.wikipedia.org/wiki/Bloom_filter[Bloom Filter] +for the lookup terms. To enable the use of the bloom filter, set any of +the following configuration options on the `bloom_filter` option: + +[horizontal] +`expected_insertions`:: + The expected number of terms to be inserted into the bloom filter. This + value must be greater than 0 and is REQUIRED. + +`fpp`:: + The false positive probability. This is the acceptable percentage of + terms that can potentially be considered a valid lookup term even though + it was not found in any documents matching the lookup query. This value + must be between 0 and 1 and defaults to 0.03 (3%). + +`hash_functions`:: + The number of times a value should be hashed before being inserted into the + bloom filter. This value must be between 1 and 255 and by default has an + optimal value calculated based on the `expected_insertions` and `fpp`. + +The optimal bloom filter configuration is very dependent on the number of terms +gathered from matching documents and the number of terms the filter will actually +compare against the bloom filter. For a higher precision (less false positives) +you can increase the number of `expected_insertions`, lower the `fpp`, or increase +the number of `hash_functions`. As you get a higher precision your response times +will get slower due to resulting bloom filter getting larger and/or using more CPU +to calculate the hashes. Increasing the `fpp` value is typically the only thing +required to get faster response times. + +The bloom filter support is an advanced feature and will require some trial and error +to find optimal values. + ["float",id="query-dsl-terms-filter-lookup-caching"] ==== Terms lookup caching @@ -143,13 +249,13 @@ There is an additional cache involved, which caches the lookup of the lookup document to the actual terms. This lookup cache is a LRU cache. This cache has the following options: -`indices.cache.filter.terms.size`:: +`indices.cache.filter.terms.size`:: The size of the lookup cache. The default is `10mb`. -`indices.cache.filter.terms.expire_after_access`:: +`indices.cache.filter.terms.expire_after_access`:: The time after the last read an entry should expire. Disabled by default. -`indices.cache.filter.terms.expire_after_write`:: +`indices.cache.filter.terms.expire_after_write`:: The time after the last write an entry should expire. Disabled by default. All options for the lookup of the documents cache can only be configured @@ -225,3 +331,96 @@ curl -XPUT localhost:9200/users/user/2 -d '{ -------------------------------------------------- In which case, the lookup path will be `followers.id`. + +[float] +==== Terms lookup by query example + +In the following example we are replicating the +"<>" by looking up the +"pid" values from children documents with the tag "something" and +then filtering only parent documents that have an "id" matching one +of the children's "pid" values. + +In this example, parents and children are stored in their own indices. + +[source,js] +-------------------------------------------------- +curl -XPOST 'http://localhost:9200/parentIndex/_search' -d '{ + "query": { + "constant_score": { + "filter": { + "terms": { + "id": { + "index": "childIndex", + "type": "childType", + "path": "pid", + "filter": { + "term": { + "tag": "something" + } + } + } + } + } + } + } +}' +-------------------------------------------------- + +Using the "<>" support: + +[source,js] +-------------------------------------------------- +curl -XPOST 'http://localhost:9200/parentIndex/_search' -d '{ + "query": { + "constant_score": { + "filter": { + "terms": { + "id": { + "index": "childIndex", + "type": "childType", + "path": "pid", + "filter": { + "term": { + "tag": "something" + } + }, + "bloom_filter": { + "expected_insertions": 10000 + } + } + } + } + } + } +}' +-------------------------------------------------- + +Here is another example where we are searching for products or services +mentioning "elasticsearch". Products, Services, and Companies are all stored +in their own index and contain a numeric "company_id" field. Both products +and services have a "description" field. + +[source,js] +-------------------------------------------------- +curl -XPOST 'http://localhost:9200/companies/_search' -d '{ + "query": { + "constant_score": { + "filter": { + "terms": { + "company_id": { + "indices": ["products", "services"], + "path": "company_id", + "filter": { + "term": { + "description": "elasticsearch" + } + } + } + } + } + } + } +}' +-------------------------------------------------- + diff --git a/src/main/java/org/elasticsearch/action/ActionModule.java b/src/main/java/org/elasticsearch/action/ActionModule.java index 5dfec1fccde6b..21e8459b5732d 100644 --- a/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/src/main/java/org/elasticsearch/action/ActionModule.java @@ -19,6 +19,9 @@ package org.elasticsearch.action; +import org.elasticsearch.action.terms.TermsByQueryAction; +import org.elasticsearch.action.terms.TransportTermsByQueryAction; + import com.google.common.collect.Maps; import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.cluster.health.TransportClusterHealthAction; @@ -258,6 +261,7 @@ protected void configure() { registerAction(DeleteAction.INSTANCE, TransportDeleteAction.class, TransportIndexDeleteAction.class, TransportShardDeleteAction.class); registerAction(CountAction.INSTANCE, TransportCountAction.class); + registerAction(TermsByQueryAction.INSTANCE, TransportTermsByQueryAction.class); registerAction(SuggestAction.INSTANCE, TransportSuggestAction.class); registerAction(UpdateAction.INSTANCE, TransportUpdateAction.class); registerAction(MultiGetAction.INSTANCE, TransportMultiGetAction.class, diff --git a/src/main/java/org/elasticsearch/action/terms/ResponseTerms.java b/src/main/java/org/elasticsearch/action/terms/ResponseTerms.java new file mode 100644 index 0000000000000..12fb720a7dfc5 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/ResponseTerms.java @@ -0,0 +1,977 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import com.carrotsearch.hppc.DoubleOpenHashSet; +import com.carrotsearch.hppc.LongOpenHashSet; +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.action.terms.TransportTermsByQueryAction.HitSetCollector; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BloomFilter; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.index.fielddata.*; + +import java.io.IOException; +import java.util.List; + +/** + * Gathers and stores terms for a {@link TermsByQueryRequest}. + */ +public abstract class ResponseTerms implements Streamable { + + protected transient final HitSetCollector collector; + + /** + * Default constructor + */ + ResponseTerms() { + this.collector = null; + } + + /** + * Constructor used before term collection + * + * @param collector the collector used during the lookup query execution + */ + public ResponseTerms(HitSetCollector collector) { + this.collector = collector; + } + + /** + * Deserialize to correct {@link ResponseTerms} implementation. + */ + public static ResponseTerms deserialize(StreamInput in) throws IOException { + Type type = Type.fromId(in.readByte()); + ResponseTerms rt; + switch (type) { + case LONGS: + rt = new LongsResponseTerms(); + break; + case DOUBLES: + rt = new DoublesResponseTerms(); + break; + case BLOOM: + rt = new BloomResponseTerms(); + break; + default: + rt = new BytesResponseTerms(); + } + + rt.readFrom(in); + return rt; + } + + /** + * Serialize a {@link ResponseTerms}. + */ + public static void serialize(ResponseTerms rt, StreamOutput out) throws IOException { + out.writeByte(rt.getType().id()); + rt.writeTo(out); + } + + /** + * Creates a new {@link ResponseTerms} based on the fielddata type and request settings. Returns a + * {@link LongsResponseTerms} for non-floating point numeric fields, {@link DoublesResponseTerms} for floating point + * numeric fields, and a {@link BytesResponseTerms} for all other field types. If + * {@link TermsByQueryRequest#useBloomFilter} is set, then a {@link BloomResponseTerms} is returned for + * non-numeric fields. + * + * @param collector the collector used during the lookup query execution + * @param indexFieldData the fielddata for the lookup field + * @param request the lookup request + * @return {@link ResponseTerms} for the fielddata type + */ + public static ResponseTerms get(HitSetCollector collector, IndexFieldData indexFieldData, TermsByQueryRequest request) { + if (indexFieldData instanceof IndexNumericFieldData) { + IndexNumericFieldData numFieldData = (IndexNumericFieldData) indexFieldData; + if (numFieldData.getNumericType().isFloatingPoint()) { + return new DoublesResponseTerms(collector, numFieldData, request.maxTermsPerShard()); + } else { + return new LongsResponseTerms(collector, numFieldData, request.maxTermsPerShard()); + } + } else { + // use bytes or bloom for all non-numeric fields types + if (request.useBloomFilter()) { + BloomResponseTerms bloomTerms = + new BloomResponseTerms(collector, indexFieldData, request.bloomFpp(), + request.bloomExpectedInsertions(), request.bloomHashFunctions(), + request.maxTermsPerShard()); + return bloomTerms; + } else { + return new BytesResponseTerms(collector, indexFieldData, request.maxTermsPerShard()); + } + } + } + + /** + * Gets a {@link ResponseTerms} for a specified type and initial size. + * + * @param type The {@link ResponseTerms.Type} to return + * @param size The number of expected terms. + * @return {@link ResponseTerms} of the specified type. + */ + public static ResponseTerms get(Type type, int size) { + switch (type) { + case LONGS: + return new LongsResponseTerms(size); + case DOUBLES: + return new DoublesResponseTerms(size); + case BLOOM: + return new BloomResponseTerms(); + default: + return new BytesResponseTerms(size); + } + } + + /** + * Called before gathering terms on a new {@link AtomicReaderContext}. Should be used to perform operations such + * as loading fielddata, etc. + * + * @param context The {@link AtomicReaderContext} we are about to process + */ + protected abstract void load(AtomicReaderContext context); + + /** + * Called for each hit in the current {@link AtomicReaderContext}. Should be used to extract terms. + * + * @param docId The internal lucene docid for the hit in the current {@link AtomicReaderContext}. + */ + protected abstract void processDoc(int docId); + + /** + * Returns the type. + * + * @return The {@link ResponseTerms.Type} + */ + public abstract Type getType(); + + /** + * Called to merging {@link ResponseTerms} from other shards. + * + * @param other The {@link ResponseTerms} to merge with + */ + public abstract void merge(ResponseTerms other); + + /** + * The number of terms in the {@link ResponseTerms}. + * + * @return The number of terms + */ + public abstract long size(); + + /** + * The size of the {@link ResponseTerms} in bytes. + * + * @return The size in bytes + */ + public abstract long getSizeInBytes(); + + /** + * Returns the the terms. + * + * @return The terms + */ + public abstract Object getTerms(); + + /** + * Returns if the max number of terms has been gathered or not. + * + * @return true if we have hit the max number of terms, false otherwise. + */ + public abstract boolean isFull(); + + /** + * Process the terms lookup query. + * + * @param leaves a list of {@link AtomicReaderContext} the lookup query was executed against. + * @throws IOException + */ + public void process(List leaves) throws IOException { + FixedBitSet[] bitSets = collector.getFixedSets(); + for (int i = 0; i < leaves.size(); i++) { + AtomicReaderContext readerContext = leaves.get(i); + load(readerContext); + DocIdSetIterator iterator = bitSets[i].iterator(); + int docId = 0; + while ((docId = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS && !isFull()) { + processDoc(docId); + } + } + } + + /** + * The various types of {@link ResponseTerms}. + */ + public static enum Type { + BYTES((byte) 0), + LONGS((byte) 1), + DOUBLES((byte) 2), + BLOOM((byte) 3); + + private final byte id; + + Type(byte id) { + this.id = id; + } + + public byte id() { + return id; + } + + public static Type fromId(byte id) { + switch (id) { + case 0: + return BYTES; + case 1: + return LONGS; + case 2: + return DOUBLES; + case 3: + return BLOOM; + default: + return BYTES; + } + } + } + + /** + * A {@link ResponseTerms} implementation that creates a {@link BloomFilter} while collecting terms. The {@link BloomFilter} + * is less expensive to serialize over the network resulting in much better response times. The trade off is the + * possibility of false positives. The {@link BloomFilter} can be tuned to for size (faster) or accuracy (slower) depending + * on the user's needs. + */ + public static class BloomResponseTerms extends ResponseTerms { + + private transient final IndexFieldData indexFieldData; + private transient BytesValues values; + private BloomFilter bloomFilter; + private long maxTerms = Long.MAX_VALUE; // max number of terms to gather per shard + private long size = 0; + + /** + * Default constructor + */ + BloomResponseTerms() { + this.indexFieldData = null; + } + + /** + * Constructor to be used before term collection and to specify custom {@link BloomFilter} settings. + * + * @param collector the collector used during terms lookup query + * @param indexFieldData the fielddata for the lookup field + * @param fpp false positive probability, must be between value between 0 and 1. + * @param expectedInsertions the expected number of terms to be inserted into the bloom filter + * @param numHashFunctions the number of hash functions to use + */ + BloomResponseTerms(HitSetCollector collector, IndexFieldData indexFieldData, + Double fpp, Integer expectedInsertions, Integer numHashFunctions, Long maxTerms) { + super(collector); + this.indexFieldData = indexFieldData; + + if (maxTerms != null) { + this.maxTerms = maxTerms; + } + + if (fpp == null) { + fpp = 0.03; + } + + if (expectedInsertions == null) { + expectedInsertions = 100; + } + + if (numHashFunctions != null) { + bloomFilter = BloomFilter.create(expectedInsertions, fpp, numHashFunctions); + } else { + bloomFilter = BloomFilter.create(expectedInsertions, fpp); + } + } + + /** + * Load the fielddata + * + * @param context The {@link AtomicReaderContext} we are about to process + */ + @Override + protected void load(AtomicReaderContext context) { + values = indexFieldData.load(context).getBytesValues(false); // load field data cache + } + + /** + * Extracts all values from the fielddata for the lookup field and inserts them in the bloom filter. + * + * @param docId The internal lucene docid for the hit in the current {@link AtomicReaderContext}. + */ + @Override + protected void processDoc(int docId) { + final int numVals = values.setDocument(docId); + for (int i = 0; i < numVals && !isFull(); i++) { + final BytesRef term = values.nextValue(); + bloomFilter.put(term); + size += 1; + } + } + + /** + * The type. + * + * @return {@link BloomResponseTerms.Type#BLOOM} + */ + @Override + public Type getType() { + return Type.BLOOM; + } + + /** + * Merge with another {@link BloomResponseTerms} + * + * @param other The {@link ResponseTerms} to merge with + */ + @Override + public void merge(ResponseTerms other) { + assert other.getType() == Type.BLOOM; // must be a bloom + + BloomResponseTerms ot = (BloomResponseTerms) other; + size += ot.size; + + if (bloomFilter == null) { + bloomFilter = ot.bloomFilter; + } else if (ot.bloomFilter != null) { + bloomFilter.merge(ot.bloomFilter); + } + } + + /** + * The number of terms represented in the bloom filter. + * + * @return the number of terms. + */ + @Override + public long size() { + return size; + } + + /** + * The size of the bloom filter. + * + * @return The size of the bloom filter in bytes. + */ + @Override + public long getSizeInBytes() { + return bloomFilter != null ? bloomFilter.getSizeInBytes() : 0; + } + + /** + * Returns the bloom filter. + * + * @return the {@link BloomFilter} + */ + @Override + public Object getTerms() { + return bloomFilter; + } + + /** + * Returns if the max number of terms has been gathered or not. + * + * @return true if we have hit the max number of terms, false otherwise. + */ + @Override + public boolean isFull() { + return size == maxTerms; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + maxTerms = in.readVLong(); + if (in.readBoolean()) { + bloomFilter = BloomFilter.readFrom(in); + } + + size = in.readVLong(); + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(maxTerms); + if (bloomFilter != null) { + out.writeBoolean(true); + BloomFilter.writeTo(bloomFilter, out); + } else { + out.writeBoolean(false); + } + + out.writeVLong(size); + } + } + + /** + * A {@link ResponseTerms} implementation for non-numeric fields that collects all terms into a set. All terms + * are serialized and transferred over the network which could lead to slower response times if performing a lookup + * on a field with a high-cardinality and a large result set. + */ + public static class BytesResponseTerms extends ResponseTerms { + + private transient final IndexFieldData indexFieldData; + private long maxTerms = Long.MAX_VALUE; // max number of terms to gather per shard + private long sizeInBytes; + private transient BytesValues values; + private BytesRefHash termsHash; + + /** + * Default constructor + */ + BytesResponseTerms() { + this.indexFieldData = null; + } + + /** + * Constructor to be used before merging. Initializes the set with the correct size to avoid rehashing during + * the merge. + * + * @param size the number of terms that will be in the resulting set + */ + BytesResponseTerms(long size) { + this.termsHash = new BytesRefHash(size, BigArrays.NON_RECYCLING_INSTANCE); + this.indexFieldData = null; + } + + /** + * Constructor to be used for term collection on each shard. + * + * @param collector the collector used during the lookup query execution + * @param indexFieldData the fielddata for the lookup field. + */ + BytesResponseTerms(HitSetCollector collector, IndexFieldData indexFieldData, Long maxTerms) { + super(collector); + if (maxTerms != null) { + this.maxTerms = maxTerms; + } + + long collectorHits = collector.getHits(); + this.indexFieldData = indexFieldData; + // TODO: use collectorHits to init? + this.termsHash = new BytesRefHash(this.maxTerms < collectorHits ? this.maxTerms : collectorHits, + BigArrays.NON_RECYCLING_INSTANCE); + } + + /** + * Load the fielddata. + * + * @param context The {@link AtomicReaderContext} we are about to process + */ + @Override + protected void load(AtomicReaderContext context) { + values = indexFieldData.load(context).getBytesValues(true); // load field data cache + hashes + } + + /** + * Extracts all values from the fielddata for the lookup field and inserts them in the terms set. + * + * @param docId The internal lucene docid for the hit in the current {@link AtomicReaderContext}. + */ + @Override + protected void processDoc(int docId) { + final int numVals = values.setDocument(docId); + for (int i = 0; i < numVals && !isFull(); i++) { + final BytesRef term = values.nextValue(); + termsHash.add(term, values.currentValueHash()); + // offset int + length int + object pointer + object header + array pointer + array header = 64 + sizeInBytes += term.length + 64; + } + } + + /** + * Merge with another {@link BytesResponseTerms} + * + * @param other The {@link ResponseTerms} to merge with + */ + @Override + public void merge(ResponseTerms other) { + assert other.getType() == Type.BYTES; + BytesResponseTerms ot = (BytesResponseTerms) other; + sizeInBytes += ot.sizeInBytes; // update size + if (termsHash == null) { + // probably never hit this since we init terms to known size before merge + termsHash = new BytesRefHash(ot.size(), BigArrays.NON_RECYCLING_INSTANCE); + } + + // TODO: maybe make it an option not to merge? + BytesRef spare = new BytesRef(); + for (long i = 0; i < ot.termsHash.size(); i++) { + ot.termsHash.get(i, spare); + // TODO: avoid rehash of hashCode by pulling out of ot.termsHash somehow? + termsHash.add(spare, spare.hashCode()); + } + } + + /** + * The type. + * + * @return {@link ResponseTerms.Type#BYTES} + */ + @Override + public Type getType() { + return Type.BYTES; + } + + + /** + * Returns if the max number of terms has been gathered or not. + * + * @return true if we have hit the max number of terms, false otherwise. + */ + @Override + public boolean isFull() { + return termsHash.size() == maxTerms; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + long size = in.readVLong(); + maxTerms = in.readVLong(); + sizeInBytes = in.readVLong(); + termsHash = new BytesRefHash(size, BigArrays.NON_RECYCLING_INSTANCE); + for (long i = 0; i < size; i++) { + BytesRef term = in.readBytesRef(); + termsHash.add(term, term.hashCode()); + } + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(termsHash.size()); + out.writeVLong(maxTerms); + out.writeVLong(sizeInBytes); + BytesRef spare = new BytesRef(); + for (long i = 0; i < termsHash.size(); i++) { + termsHash.get(i, spare); + out.writeBytesRef(spare); + } + } + + /** + * The number of collected terms. + * + * @return the number of terms. + */ + @Override + public long size() { + return termsHash.size(); + } + + /** + * The size of the terms. + * + * @return the size in bytes. + */ + @Override + public long getSizeInBytes() { + return sizeInBytes; + } + + /** + * Returns the terms. + * + * @return {@link org.elasticsearch.common.util.BytesRefHash} + */ + @Override + public Object getTerms() { + return termsHash; + } + } + + /** + * A {@link ResponseTerms} implementation for non-floating point numeric fields that collects all terms into a set. + * All terms are gathered and serialized as primitive longs. + */ + public static class LongsResponseTerms extends ResponseTerms { + + private transient final IndexNumericFieldData indexFieldData; + private transient LongValues values; + private long maxTerms = Long.MAX_VALUE; // max number of terms to gather per shard + private LongHash termsHash; + + /** + * Default constructor + */ + LongsResponseTerms() { + this.indexFieldData = null; + } + + /** + * Constructor to be used before merging. Initializes the set with the correct size to avoid rehashing during + * the merge. + * + * @param size the number of terms that will be in the resulting set + */ + LongsResponseTerms(long size) { + // create terms set, size is known so adjust for load factor so no rehashing is needed + this.termsHash = new LongHash(size, BigArrays.NON_RECYCLING_INSTANCE); + this.indexFieldData = null; + } + + /** + * Constructor to be used for term collection on each shard. + * + * @param collector the collector used during the lookup query execution + * @param indexFieldData the fielddata for the lookup field. + */ + LongsResponseTerms(HitSetCollector collector, IndexNumericFieldData indexFieldData, Long maxTerms) { + super(collector); + if (maxTerms != null) { + this.maxTerms = maxTerms; + } + + long collectorHits = collector.getHits(); + this.indexFieldData = indexFieldData; + this.termsHash = new LongHash(this.maxTerms < collectorHits ? this.maxTerms : collectorHits, + BigArrays.NON_RECYCLING_INSTANCE); + } + + /** + * Load the fielddata + * + * @param context The {@link AtomicReaderContext} we are about to process + */ + @Override + protected void load(AtomicReaderContext context) { + values = indexFieldData.load(context).getLongValues(); // load field data cache + } + + /** + * Extracts all values from the fielddata for the lookup field and inserts them in the terms set. + * + * @param docId The internal lucene docid for the hit in the current {@link AtomicReaderContext}. + */ + @Override + protected void processDoc(int docId) { + final int numVals = values.setDocument(docId); + for (int i = 0; i < numVals && !isFull(); i++) { + final long term = values.nextValue(); + termsHash.add(term); + } + } + + /** + * Merge with another {@link LongsResponseTerms}. + * + * @param other The {@link ResponseTerms} to merge with + */ + @Override + public void merge(ResponseTerms other) { + assert other.getType() == Type.LONGS; + LongsResponseTerms ot = (LongsResponseTerms) other; + + if (termsHash == null) { + // probably never hit this since we init terms to known size before merge + termsHash = new LongHash(ot.size(), BigArrays.NON_RECYCLING_INSTANCE); + } + + // TODO: maybe make it an option not to merge? + for (long i = 0; i < ot.termsHash.size(); i++) { + termsHash.add(ot.termsHash.get(i)); + } + } + + /** + * The type + * + * @return {@link ResponseTerms.Type#LONGS} + */ + @Override + public Type getType() { + return Type.LONGS; + } + + /** + * Returns if the max number of terms has been gathered or not. + * + * @return true if we have hit the max number of terms, false otherwise. + */ + @Override + public boolean isFull() { + return termsHash.size() == maxTerms; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + maxTerms = in.readVLong(); + long size = in.readVLong(); + termsHash = new LongHash(size, BigArrays.NON_RECYCLING_INSTANCE); + for (long i = 0; i < size; i++) { + termsHash.add(in.readLong()); + } + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(maxTerms); + out.writeVLong(termsHash.size()); + for (long i = 0; i < termsHash.size(); i++) { + out.writeLong(termsHash.get(i)); + } + } + + /** + * The number of terms. + * + * @return the number of terms collected. + */ + @Override + public long size() { + return termsHash.size(); + } + + /** + * The size of the gathered terms + * + * @return The size in bytes + */ + @Override + public long getSizeInBytes() { + return termsHash.size() * 8; + } + + /** + * Returns the terms. + * + * @return {@link LongOpenHashSet} + */ + @Override + public Object getTerms() { + return termsHash; + } + } + + /** + * A {@link ResponseTerms} implementation for floating point numeric fields that collects all terms into a set. + * All terms are gathered and serialized as primitive doubles. + */ + public static class DoublesResponseTerms extends ResponseTerms { + + private transient final IndexNumericFieldData indexFieldData; + private transient DoubleValues values; + private long maxTerms = Long.MAX_VALUE; // max number of terms to gather per shard + private LongHash termsHash; + + /** + * Default constructor + */ + DoublesResponseTerms() { + this.indexFieldData = null; + } + + /** + * Constructor to be used before merging. Initializes the set with the correct size to avoid rehashing during + * the merge. + * + * @param size the number of terms that will be in the resulting set + */ + DoublesResponseTerms(long size) { + this.termsHash = new LongHash(size, BigArrays.NON_RECYCLING_INSTANCE); + this.indexFieldData = null; + } + + /** + * Constructor to be used for term collection on each shard. + * + * @param collector the collector used during the lookup query execution + * @param indexFieldData the fielddata for the lookup field. + */ + DoublesResponseTerms(HitSetCollector collector, IndexNumericFieldData indexFieldData, Long maxTerms) { + super(collector); + if (maxTerms != null) { + this.maxTerms = maxTerms; + } + + long collectorHits = collector.getHits(); + this.indexFieldData = indexFieldData; + this.termsHash = new LongHash(this.maxTerms < collectorHits ? this.maxTerms : collectorHits, + BigArrays.NON_RECYCLING_INSTANCE); + } + + /** + * Load the fielddata + * + * @param context The {@link AtomicReaderContext} we are about to process + */ + @Override + protected void load(AtomicReaderContext context) { + values = indexFieldData.load(context).getDoubleValues(); // load field data cache + } + + /** + * Extracts all values from the fielddata for the lookup field and inserts them in the terms set. + * + * @param docId The internal lucene docid for the hit in the current {@link AtomicReaderContext}. + */ + @Override + protected void processDoc(int docId) { + final int numVals = values.setDocument(docId); + for (int i = 0; i < numVals && !isFull(); i++) { + final double term = values.nextValue(); + final long longTerm = Double.doubleToLongBits(term); + termsHash.add(longTerm); + } + } + + /** + * Merge with another {@link DoublesResponseTerms} + * + * @param other The {@link ResponseTerms} to merge with + */ + @Override + public void merge(ResponseTerms other) { + assert other.getType() == Type.DOUBLES; + DoublesResponseTerms ot = (DoublesResponseTerms) other; + + if (termsHash == null) { + // probably never hit this since we init terms to known size before merge + termsHash = new LongHash(ot.size(), BigArrays.NON_RECYCLING_INSTANCE); + } + + // TODO: maybe make it an option not to merge? + for (long i = 0; i < ot.termsHash.size(); i++) { + termsHash.add(ot.termsHash.get(i)); + } + } + + /** + * The type + * + * @return {@link ResponseTerms.Type#DOUBLES} + */ + @Override + public Type getType() { + return Type.DOUBLES; + } + + @Override + public boolean isFull() { + return termsHash.size() == maxTerms; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + maxTerms = in.readVLong(); + long size = in.readVLong(); + termsHash = new LongHash(size, BigArrays.NON_RECYCLING_INSTANCE); + for (long i = 0; i < size; i++) { + termsHash.add(in.readLong()); + } + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(maxTerms); + out.writeVLong(termsHash.size()); + for (long i = 0; i < termsHash.size(); i++) { + out.writeLong(termsHash.get(i)); + } + } + + /** + * The number of collected terms + * + * @return the number of terms. + */ + @Override + public long size() { + return termsHash.size(); + } + + /** + * The size of the collected terms + * + * @return The size of the terms in bytes. + */ + @Override + public long getSizeInBytes() { + return termsHash.size() * 8; + } + + /** + * Returns the terms. + * + * @return {@link DoubleOpenHashSet} + */ + @Override + public Object getTerms() { + return termsHash; + } + } +} diff --git a/src/main/java/org/elasticsearch/action/terms/ShardTermsByQueryRequest.java b/src/main/java/org/elasticsearch/action/terms/ShardTermsByQueryRequest.java new file mode 100644 index 0000000000000..df227771c3822 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/ShardTermsByQueryRequest.java @@ -0,0 +1,111 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationRequest; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Internal terms by query request executed directly against a specific index shard. + */ +class ShardTermsByQueryRequest extends BroadcastShardOperationRequest { + + @Nullable + private String[] filteringAliases; + private TermsByQueryRequest request; + + /** + * Default constructor + */ + ShardTermsByQueryRequest() { + } + + /** + * Main Constructor + * + * @param index the index of the shard request + * @param shardId the id of the shard the request is for + * @param filteringAliases optional aliases + * @param request the original {@link TermsByQueryRequest} + */ + public ShardTermsByQueryRequest(String index, int shardId, @Nullable String[] filteringAliases, TermsByQueryRequest request) { + super(index, shardId, request); + this.filteringAliases = filteringAliases; + this.request = request; + } + + /** + * Gets the filtering aliases + * + * @return the filtering aliases + */ + public String[] filteringAliases() { + return filteringAliases; + } + + /** + * Gets the original {@link TermsByQueryRequest} + * + * @return the request + */ + public TermsByQueryRequest request() { + return request; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + request = new TermsByQueryRequest(); + request.readFrom(in); + + if (in.readBoolean()) { + filteringAliases = in.readStringArray(); + } + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + request.writeTo(out); + + if (filteringAliases == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeStringArray(filteringAliases); + } + } +} diff --git a/src/main/java/org/elasticsearch/action/terms/ShardTermsByQueryResponse.java b/src/main/java/org/elasticsearch/action/terms/ShardTermsByQueryResponse.java new file mode 100644 index 0000000000000..5ffb51fa3e2ad --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/ShardTermsByQueryResponse.java @@ -0,0 +1,85 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import org.elasticsearch.action.support.broadcast.BroadcastShardOperationResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Internal terms by query response of a shard terms by query request executed directly against a specific shard. + */ +class ShardTermsByQueryResponse extends BroadcastShardOperationResponse { + + private ResponseTerms responseTerms; + + /** + * Default constructor + */ + ShardTermsByQueryResponse() { + } + + /** + * Main constructor + * + * @param index the index the request executed against + * @param shardId the id of the shard the request executed on + * @param responseTerms the terms gathered from the shard + */ + public ShardTermsByQueryResponse(String index, int shardId, ResponseTerms responseTerms) { + super(index, shardId); + this.responseTerms = responseTerms; + } + + /** + * Gets the gathered terms. + * + * @return the {@link ResponseTerms} + */ + public ResponseTerms getResponseTerms() { + return this.responseTerms; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + responseTerms = ResponseTerms.deserialize(in); + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + ResponseTerms.serialize(responseTerms, out); + } +} diff --git a/src/main/java/org/elasticsearch/action/terms/TermsByQueryAction.java b/src/main/java/org/elasticsearch/action/terms/TermsByQueryAction.java new file mode 100644 index 0000000000000..8b0a534edaafe --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/TermsByQueryAction.java @@ -0,0 +1,78 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import org.elasticsearch.action.Action; +import org.elasticsearch.client.Client; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.transport.TransportRequestOptions; + +/** + * The action to request terms by query + */ +public class TermsByQueryAction extends Action { + + public static final TermsByQueryAction INSTANCE = new TermsByQueryAction(); + public static final String NAME = "termsbyquery"; + + /** + * Default constructor + */ + private TermsByQueryAction() { + super(NAME); + } + + /** + * Gets a new {@link TermsByQueryResponse} object + * + * @return the new {@link TermsByQueryResponse}. + */ + @Override + public TermsByQueryResponse newResponse() { + return new TermsByQueryResponse(); + } + + /** + * Set transport options specific to a terms by query request. + * + * @param settings node settings + * @return the request options. + */ + @Override + public TransportRequestOptions transportOptions(Settings settings) { + TransportRequestOptions opts = new TransportRequestOptions(); + opts.withType(TransportRequestOptions.Type.BULK); // TODO: just stick with default of REG? + opts.withCompress(true); + + // return TransportRequestOptions.EMPTY; + return opts; + } + + /** + * Get a new {@link TermsByQueryRequestBuilder} + * + * @param client the client responsible for executing the request. + * @return the new {@link TermsByQueryRequestBuilder} + */ + @Override + public TermsByQueryRequestBuilder newRequestBuilder(Client client) { + return new TermsByQueryRequestBuilder(client); + } +} diff --git a/src/main/java/org/elasticsearch/action/terms/TermsByQueryRequest.java b/src/main/java/org/elasticsearch/action/terms/TermsByQueryRequest.java new file mode 100644 index 0000000000000..e59ec227bfd15 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/TermsByQueryRequest.java @@ -0,0 +1,460 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.index.query.FilterBuilder; + +import java.io.IOException; +import java.util.Arrays; + +/** + * A request to get the values from a specific field for documents matching a specific query. + *

+ * The request requires the filter source to be set either using {@link #filter(org.elasticsearch.index.query.FilterBuilder)}, or + * {@link #filter(byte[])}. + * + * @see TermsByQueryResponse + */ +public class TermsByQueryRequest extends BroadcastOperationRequest { + + @Nullable + protected String routing; + private long nowInMillis; + private Float minScore; + @Nullable + private String preference; + private BytesReference filterSource; + private boolean filterSourceUnsafe; + @Nullable + private String[] types = Strings.EMPTY_ARRAY; + private String field; + private boolean useBloomFilter = false; + private Double bloomFpp; // false positive probability + private Integer bloomExpectedInsertions; + private Integer bloomHashFunctions; + private Long maxTermsPerShard; + + TermsByQueryRequest() { + } + + /** + * Constructs a new terms by query request against the provided indices. No indices provided means it will run against all indices. + */ + public TermsByQueryRequest(String... indices) { + super(indices); + } + + /** + * Validates the request + * + * @return null if valid, exception otherwise + */ + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = super.validate(); + return validationException; + } + + /** + * Makes the filter source safe before the request is executed. + */ + @Override + protected void beforeStart() { + if (filterSource != null && filterSourceUnsafe) { + filterSource = filterSource.copyBytesArray(); + filterSourceUnsafe = false; + } + } + + /** + * The minimum score of the documents to include in the terms by query request. + */ + public Float minScore() { + return minScore; + } + + /** + * The minimum score of the documents to include in the terms by query. Defaults to null which means all + * documents will be included in the terms by query request. + */ + public TermsByQueryRequest minScore(float minScore) { + this.minScore = minScore; + return this; + } + + /** + * The field to extract terms from. + */ + public String field() { + return field; + } + + /** + * The field to extract terms from. + */ + public TermsByQueryRequest field(String field) { + this.field = field; + return this; + } + + /** + * The filter source to execute. + */ + public BytesReference filterSource() { + return filterSource; + } + + /** + * The filter source to execute. + * + * @see {@link org.elasticsearch.index.query.FilterBuilders} + */ + public TermsByQueryRequest filter(FilterBuilder filterBuilder) { + this.filterSource = filterBuilder.buildAsBytes(); + this.filterSourceUnsafe = false; + return this; + } + + /** + * The filter source to execute. + */ + public TermsByQueryRequest filter(XContentBuilder builder) { + this.filterSource = builder.bytes(); + this.filterSourceUnsafe = false; + return this; + } + + /** + * The filter source to execute. It is preferable to use {@link #filter(byte[])} + */ + public TermsByQueryRequest filter(String filterSource) { + this.filterSource = new BytesArray(filterSource); + this.filterSourceUnsafe = false; + return this; + } + + /** + * The filter source to execute. + */ + public TermsByQueryRequest filter(byte[] filterSource) { + return filter(filterSource, 0, filterSource.length, false); + } + + /** + * The filter source to execute. + */ + public TermsByQueryRequest filter(byte[] filterSource, int offset, int length, boolean unsafe) { + return filter(new BytesArray(filterSource, offset, length), unsafe); + } + + /** + * The filter source to execute. + */ + public TermsByQueryRequest filter(BytesReference filterSource, boolean unsafe) { + this.filterSource = filterSource; + this.filterSourceUnsafe = unsafe; + return this; + } + + /** + * The types of documents the query will run against. Defaults to all types. + */ + public String[] types() { + return this.types; + } + + /** + * The types of documents the query will run against. Defaults to all types. + */ + public TermsByQueryRequest types(String... types) { + this.types = types; + return this; + } + + /** + * A comma separated list of routing values to control the shards the search will be executed on. + */ + public String routing() { + return this.routing; + } + + /** + * A comma separated list of routing values to control the shards the search will be executed on. + */ + public TermsByQueryRequest routing(String routing) { + this.routing = routing; + return this; + } + + /** + * The current time in milliseconds + */ + public long nowInMillis() { + return nowInMillis; + } + + /** + * Sets the current time in milliseconds + */ + public TermsByQueryRequest nowInMillis(long nowInMillis) { + this.nowInMillis = nowInMillis; + return this; + } + + /** + * The routing values to control the shards that the request will be executed on. + */ + public TermsByQueryRequest routing(String... routings) { + this.routing = Strings.arrayToCommaDelimitedString(routings); + return this; + } + + /** + * The preference value to control what node the request will be executed on + */ + public TermsByQueryRequest preference(String preference) { + this.preference = preference; + return this; + } + + /** + * The current preference value + */ + public String preference() { + return this.preference; + } + + /** + * If the bloom filter should be used. + */ + public TermsByQueryRequest useBloomFilter(boolean useBloomFilter) { + this.useBloomFilter = useBloomFilter; + return this; + } + + /** + * If the bloom filter will be used. + */ + public boolean useBloomFilter() { + return useBloomFilter; + } + + /** + * The bloom filter false positive probability + */ + public TermsByQueryRequest bloomFpp(double bloomFpp) { + this.bloomFpp = bloomFpp; + return this; + } + + /** + * The bloom filter false positive probability + */ + public Double bloomFpp() { + return bloomFpp; + } + + /** + * The expected insertions size for the bloom filter + */ + public TermsByQueryRequest bloomExpectedInsertions(int bloomExpectedInsertions) { + this.bloomExpectedInsertions = bloomExpectedInsertions; + return this; + } + + /** + * The expected insertions for the bloom filter + */ + public Integer bloomExpectedInsertions() { + return bloomExpectedInsertions; + } + + /** + * The number of hash functions for the bloom filter + */ + public TermsByQueryRequest bloomHashFunctions(int bloomHashFunctions) { + this.bloomHashFunctions = bloomHashFunctions; + return this; + } + + /** + * The number of hash functions for the bloom filter + */ + public Integer bloomHashFunctions() { + return bloomHashFunctions; + } + + /** + * The max number of terms to gather per shard + */ + public TermsByQueryRequest maxTermsPerShard(long maxTermsPerShard) { + this.maxTermsPerShard = maxTermsPerShard; + return this; + } + + /** + * The max number of terms to gather per shard + */ + public Long maxTermsPerShard() { + return maxTermsPerShard; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + + if (in.readBoolean()) { + minScore = in.readFloat(); + } + + routing = in.readOptionalString(); + preference = in.readOptionalString(); + + filterSourceUnsafe = false; + if (in.readBoolean()) { + filterSource = in.readBytesReference(); + } else { + filterSource = null; + } + + if (in.readBoolean()) { + types = in.readStringArray(); + } + + field = in.readString(); + nowInMillis = in.readVLong(); + useBloomFilter = in.readBoolean(); + + if (in.readBoolean()) { + bloomFpp = in.readDouble(); + } + + if (in.readBoolean()) { + bloomExpectedInsertions = in.readVInt(); + } + + if (in.readBoolean()) { + bloomHashFunctions = in.readVInt(); + } + + if (in.readBoolean()) { + maxTermsPerShard = in.readVLong(); + } + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + + if (minScore == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeFloat(minScore); + } + + out.writeOptionalString(routing); + out.writeOptionalString(preference); + + if (filterSource != null) { + out.writeBoolean(true); + out.writeBytesReference(filterSource); + } else { + out.writeBoolean(false); + } + + if (types == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeStringArray(types); + } + + out.writeString(field); + out.writeVLong(nowInMillis); + out.writeBoolean(useBloomFilter); + + if (bloomFpp == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeDouble(bloomFpp); + } + + if (bloomExpectedInsertions == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeVInt(bloomExpectedInsertions); + } + + if (bloomHashFunctions == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeVInt(bloomHashFunctions); + } + + if (maxTermsPerShard == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + out.writeVLong(maxTermsPerShard); + } + } + + /** + * String representation of the request + * + * @return + */ + @Override + public String toString() { + String sSource = "_na_"; + try { + sSource = XContentHelper.convertToJson(filterSource, false); + } catch (Exception e) { + // ignore + } + return "[" + Arrays.toString(indices) + "]" + Arrays.toString(types) + ", filterSource[" + sSource + "]"; + } +} diff --git a/src/main/java/org/elasticsearch/action/terms/TermsByQueryRequestBuilder.java b/src/main/java/org/elasticsearch/action/terms/TermsByQueryRequestBuilder.java new file mode 100644 index 0000000000000..7248f15c97a11 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/TermsByQueryRequestBuilder.java @@ -0,0 +1,170 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.broadcast.BroadcastOperationRequestBuilder; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.internal.InternalClient; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.index.query.FilterBuilder; + +/** + * A terms by query action request builder. This is an internal api. + */ +public class TermsByQueryRequestBuilder extends BroadcastOperationRequestBuilder { + + public TermsByQueryRequestBuilder(Client client) { + super((InternalClient) client, new TermsByQueryRequest()); + } + + /** + * The types of documents the query will run against. Defaults to all types. + */ + public TermsByQueryRequestBuilder setTypes(String... types) { + request.types(types); + return this; + } + + /** + * The minimum score of the documents to include in the terms by query. Defaults to -1 which means all documents will be + * included in the terms by query. + */ + public TermsByQueryRequestBuilder setMinScore(float minScore) { + request.minScore(minScore); + return this; + } + + /** + * A comma separated list of routing values to control the shards the search will be executed on. + */ + public TermsByQueryRequestBuilder setRouting(String routing) { + request.routing(routing); + 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, _shards:x,y to operate on shards x & y, or a custom value, which + * guarantees that the same order will be used across different requests. + */ + public TermsByQueryRequestBuilder setPreference(String preference) { + request.preference(preference); + return this; + } + + /** + * The routing values to control the shards that the search will be executed on. + */ + public TermsByQueryRequestBuilder setRouting(String... routing) { + request.routing(routing); + return this; + } + + /** + * The field to extract terms from. + */ + public TermsByQueryRequestBuilder setField(String field) { + request.field(field); + return this; + } + + /** + * The filter source to execute. + * + * @see org.elasticsearch.index.query.FilterBuilders + */ + public TermsByQueryRequestBuilder setFilter(FilterBuilder filterBuilder) { + request.filter(filterBuilder); + return this; + } + + /** + * The filter source to execute. + */ + public TermsByQueryRequestBuilder setFilter(BytesReference filterSource) { + request.filter(filterSource, false); + return this; + } + + /** + * The filter source to execute. + */ + public TermsByQueryRequestBuilder setFilter(BytesReference filterSource, boolean unsafe) { + request.filter(filterSource, unsafe); + return this; + } + + /** + * The filter source to execute. + */ + public TermsByQueryRequestBuilder setFilter(byte[] filterSource) { + request.filter(filterSource); + return this; + } + + /** + * If we should use a bloom filter to gather terms + */ + public TermsByQueryRequestBuilder setUseBloomFilter(boolean useBloomFilter) { + request.useBloomFilter(useBloomFilter); + return this; + } + + /** + * The BloomFilter false positive probability + */ + public TermsByQueryRequestBuilder setBloomFpp(double bloomFpp) { + request.bloomFpp(bloomFpp); + return this; + } + + /** + * The expected insertions for the BloomFilter + */ + public TermsByQueryRequestBuilder setExpectedInsertions(int bloomExpectedInsertions) { + request.bloomExpectedInsertions(bloomExpectedInsertions); + return this; + } + + /** + * The number of hashes to use in the BloomFilter + */ + public TermsByQueryRequestBuilder setBloomHashFunctions(int bloomHashFunctions) { + request.bloomHashFunctions(bloomHashFunctions); + return this; + } + + /** + * The max number of terms collected per shard + */ + public TermsByQueryRequestBuilder setMaxTermsPerShard(long maxTermsPerShard) { + request.maxTermsPerShard(maxTermsPerShard); + return this; + } + + /** + * Executes the the request + */ + @Override + protected void doExecute(ActionListener listener) { + ((InternalClient) client).execute(TermsByQueryAction.INSTANCE, request, listener); + } +} diff --git a/src/main/java/org/elasticsearch/action/terms/TermsByQueryResponse.java b/src/main/java/org/elasticsearch/action/terms/TermsByQueryResponse.java new file mode 100644 index 0000000000000..beea40ab46b38 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/TermsByQueryResponse.java @@ -0,0 +1,90 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +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 java.io.IOException; +import java.util.List; + +/** + * The response of the terms by query action. + */ +public class TermsByQueryResponse extends BroadcastOperationResponse { + + private ResponseTerms responseTerms; + + /** + * Default constructor + */ + TermsByQueryResponse() { + } + + /** + * Main constructor + * + * @param responseTerms the merged terms + * @param totalShards the number of shards the request executed on + * @param successfulShards the number of shards the request executed on successfully + * @param failedShards the number of failed shards + * @param shardFailures the failures + */ + TermsByQueryResponse(ResponseTerms responseTerms, int totalShards, int successfulShards, int failedShards, + List shardFailures) { + super(totalShards, successfulShards, failedShards, shardFailures); + this.responseTerms = responseTerms; + } + + /** + * Gets the merged terms + * + * @return the terms + */ + public ResponseTerms getResponseTerms() { + return responseTerms; + } + + /** + * Deserialize + * + * @param in the input + * @throws IOException + */ + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + responseTerms = ResponseTerms.deserialize(in); + } + + /** + * Serialize + * + * @param out the output + * @throws IOException + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + ResponseTerms.serialize(responseTerms, out); + } +} diff --git a/src/main/java/org/elasticsearch/action/terms/TransportTermsByQueryAction.java b/src/main/java/org/elasticsearch/action/terms/TransportTermsByQueryAction.java new file mode 100644 index 0000000000000..88aaea79908b1 --- /dev/null +++ b/src/main/java/org/elasticsearch/action/terms/TransportTermsByQueryAction.java @@ -0,0 +1,360 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import com.google.common.collect.ImmutableMap; +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.search.*; +import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.ElasticsearchException; +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.cache.recycler.CacheRecycler; +import org.elasticsearch.cache.recycler.PageCacheRecycler; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.lucene.search.XConstantScoreQuery; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.mapper.FieldMapper; +import org.elasticsearch.index.query.ParsedQuery; +import org.elasticsearch.index.query.QueryParseContext; +import org.elasticsearch.index.service.IndexService; +import org.elasticsearch.index.shard.service.IndexShard; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.SearchContextException; +import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.internal.DefaultSearchContext; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.query.QueryPhaseExecutionException; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; +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; + +/** + * The terms by query transport operation + */ +public class TransportTermsByQueryAction extends TransportBroadcastOperationAction { + + private final IndicesService indicesService; + private final ScriptService scriptService; + private final CacheRecycler cacheRecycler; + private final PageCacheRecycler pageCacheRecycler; + private final BigArrays bigArrays; + + /** + * Constructor + */ + @Inject + public TransportTermsByQueryAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, + TransportService transportService, IndicesService indicesService, + ScriptService scriptService, CacheRecycler cacheRecycler, + PageCacheRecycler pageCacheRecycler, BigArrays bigArrays) { + super(settings, threadPool, clusterService, transportService); + this.indicesService = indicesService; + this.scriptService = scriptService; + this.cacheRecycler = cacheRecycler; + this.pageCacheRecycler = pageCacheRecycler; + this.bigArrays = bigArrays; + } + + /** + * Executes the actions. + */ + @Override + protected void doExecute(TermsByQueryRequest request, ActionListener listener) { + request.nowInMillis(System.currentTimeMillis()); // set time to be used in scripts + super.doExecute(request, listener); + } + + /** + * The threadpool this request will execute against + */ + @Override + protected String executor() { + return ThreadPool.Names.SEARCH; + } + + /** + * The action name + */ + @Override + protected String transportAction() { + return TermsByQueryAction.NAME; + } + + /** + * Creates a new {@link TermsByQueryRequest} + */ + @Override + protected TermsByQueryRequest newRequest() { + return new TermsByQueryRequest(); + } + + /** + * Creates a new {@link ShardTermsByQueryRequest} + */ + @Override + protected ShardTermsByQueryRequest newShardRequest() { + return new ShardTermsByQueryRequest(); + } + + /** + * Creates a new {@link ShardTermsByQueryRequest} + */ + @Override + protected ShardTermsByQueryRequest newShardRequest(ShardRouting shard, TermsByQueryRequest request) { + String[] filteringAliases = clusterService.state().metaData().filteringAliases(shard.index(), request.indices()); + return new ShardTermsByQueryRequest(shard.index(), shard.id(), filteringAliases, request); + } + + /** + * Creates a new {@link ShardTermsByQueryResponse} + */ + @Override + protected ShardTermsByQueryResponse newShardResponse() { + return new ShardTermsByQueryResponse(); + } + + /** + * The shards this request will execute against. + */ + @Override + protected GroupShardsIterator shards(ClusterState clusterState, TermsByQueryRequest request, String[] concreteIndices) { + Map> routingMap = clusterState.metaData().resolveSearchRouting(request.routing(), request.indices()); + return clusterService.operationRouting().searchShards(clusterState, + request.indices(), + concreteIndices, + routingMap, + request.preference()); + } + + @Override + protected ClusterBlockException checkGlobalBlock(ClusterState state, TermsByQueryRequest request) { + return state.blocks().globalBlockedException(ClusterBlockLevel.READ); + } + + @Override + protected ClusterBlockException checkRequestBlock(ClusterState state, TermsByQueryRequest request, String[] concreteIndices) { + return state.blocks().indicesBlockedException(ClusterBlockLevel.READ, concreteIndices); + } + + /** + * Merges the individual shard responses and returns the final {@link TermsByQueryResponse}. + */ + @Override + protected TermsByQueryResponse newResponse(TermsByQueryRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { + int successfulShards = 0; + int failedShards = 0; + int numTerms = 0; + ResponseTerms responseTerms = null; + ResponseTerms[] responses = new ResponseTerms[shardsResponses.length()]; + List shardFailures = null; + + // we check each shard response + 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(); + } + logger.info("shard operation failed", (BroadcastShardOperationFailedException) shardResponse); + shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse)); + } else { + // on successful shard response, just add to the array or responses so we can process them below + // we calculate the total number of terms gathered across each shard so we can use it during + // initialization of the final ResponseTerms below (to avoid rehashing during merging) + ShardTermsByQueryResponse shardResp = ((ShardTermsByQueryResponse) shardResponse); + ResponseTerms response = shardResp.getResponseTerms(); + responses[i] = response; + numTerms += response.size(); + successfulShards++; + } + } + + // merge the responses + for (int i = 0; i < responses.length; i++) { + ResponseTerms response = responses[i]; + if (response == null) { + continue; + } + + // responseTerms is responsible for the merge, use first non-null response + // set size to avoid rehashing on certain implementations. + if (responseTerms == null) { + responseTerms = ResponseTerms.get(response.getType(), numTerms); + } + + responseTerms.merge(response); + } + + return new TermsByQueryResponse(responseTerms, shardsResponses.length(), successfulShards, failedShards, shardFailures); + } + + /** + * The operation that executes the query and generates a {@link ShardTermsByQueryResponse} for each shard. + */ + @Override + protected ShardTermsByQueryResponse shardOperation(ShardTermsByQueryRequest shardRequest) throws ElasticsearchException { + IndexService indexService = indicesService.indexServiceSafe(shardRequest.index()); + IndexShard indexShard = indexService.shardSafe(shardRequest.shardId()); + TermsByQueryRequest request = shardRequest.request(); + + SearchShardTarget shardTarget = + new SearchShardTarget(clusterService.localNode().id(), shardRequest.index(), shardRequest.shardId()); + + ShardSearchRequest shardSearchRequest = new ShardSearchRequest() + .types(request.types()) + .filteringAliases(shardRequest.filteringAliases()) + .nowInMillis(request.nowInMillis()); + + SearchContext context = new DefaultSearchContext(0, shardSearchRequest, shardTarget, + indexShard.acquireSearcher("termsByQuery"), indexService, indexShard, scriptService, cacheRecycler, + pageCacheRecycler, bigArrays); + + try { + SearchContext.setCurrent(context); + FieldMapper fieldMapper = context.smartNameFieldMapper(request.field()); + if (fieldMapper == null) { + throw new SearchContextException(context, "field not found"); + } + + IndexFieldData indexFieldData = context.fieldData().getForField(fieldMapper); + + if (request.minScore() != null) { + context.minimumScore(request.minScore()); + } + + BytesReference filterSource = request.filterSource(); + if (filterSource != null && filterSource.length() > 0) { + XContentParser filterParser = null; + try { + filterParser = XContentFactory.xContent(filterSource).createParser(filterSource); + QueryParseContext.setTypes(request.types()); + Filter filter = indexService.queryParserService().parseInnerFilter(filterParser).filter(); + context.parsedQuery(new ParsedQuery(new XConstantScoreQuery(filter), ImmutableMap.of())); + } finally { + QueryParseContext.removeTypes(); + if (filterParser != null) { + filterParser.close(); + } + } + } + + context.preProcess(); + + // execute the search only gathering the hit count and bitset for each segment + HitSetCollector termCollector = new HitSetCollector(context.searcher().getTopReaderContext().leaves().size()); + Query query = context.query(); + if (!(query instanceof ConstantScoreQuery)) { + query = new ConstantScoreQuery(query); + } + + context.searcher().search(query, termCollector); + + // gather the terms reading the values from the field data cache + // the number of terms will be less than or equal to the total hits from the collector + ResponseTerms responseTerms = ResponseTerms.get(termCollector, indexFieldData, request); + responseTerms.process(context.searcher().getTopReaderContext().leaves()); + + return new ShardTermsByQueryResponse(shardRequest.index(), shardRequest.shardId(), responseTerms); + } catch (Throwable e) { + logger.info("error executing shard operation", e); + throw new QueryPhaseExecutionException(context, "failed to execute termsByQuery", e); + } finally { + // this will also release the index searcher + context.close(); + SearchContext.removeCurrent(); + } + } + + /* + * Collector that tracks the total number of hits and the BitSet for each segment. + */ + protected class HitSetCollector extends Collector { + private final FixedBitSet[] fixedBitSets; + private FixedBitSet current; + private int hits; + + public HitSetCollector(int numSegments) { + this.fixedBitSets = new FixedBitSet[numSegments]; + } + + @Override + public void collect(int doc) throws IOException { + current.set(doc); + hits = hits + 1; + } + + @Override + public void setNextReader(AtomicReaderContext context) throws IOException { + current = new FixedBitSet(context.reader().maxDoc()); + fixedBitSets[context.ord] = current; + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } + + /** + * The total hits + */ + public int getHits() { + return hits; + } + + /** + * The BitSets for each segment + */ + public FixedBitSet[] getFixedSets() { + return fixedBitSets; + } + } + +} diff --git a/src/main/java/org/elasticsearch/common/util/AbstractPagedHashMap.java b/src/main/java/org/elasticsearch/common/util/AbstractPagedHashMap.java index 0793da83f7f6b..b41044cf84ff4 100644 --- a/src/main/java/org/elasticsearch/common/util/AbstractPagedHashMap.java +++ b/src/main/java/org/elasticsearch/common/util/AbstractPagedHashMap.java @@ -32,13 +32,13 @@ abstract class AbstractPagedHashMap implements Releasable { // collisions may result into worse lookup performance. static final float DEFAULT_MAX_LOAD_FACTOR = 0.6f; - static long hash(long value) { + public static long hash(long value) { // Don't use the value directly. Under some cases eg dates, it could be that the low bits don't carry much value and we would like // all bits of the hash to carry as much value return MurmurHash3.hash(value); } - static long hash(double value) { + public static long hash(double value) { return hash(Double.doubleToLongBits(value)); } diff --git a/src/main/java/org/elasticsearch/common/util/BloomFilter.java b/src/main/java/org/elasticsearch/common/util/BloomFilter.java index 7b375bb5ec845..2147b7fb284cc 100644 --- a/src/main/java/org/elasticsearch/common/util/BloomFilter.java +++ b/src/main/java/org/elasticsearch/common/util/BloomFilter.java @@ -258,9 +258,29 @@ public long getSizeInBytes() { return bits.bitSize() + 8; } + public boolean isEmpty() { + return bits.bitCount == 0; + } + + /** + * Merge with another {@link BloomFilter}. The other bloom filter must have the same size, number of hash + * functions, and hashing type. + * + * @param other the other {@link BloomFilter} to merge with. + */ + public void merge(BloomFilter other) { + if (bits.data.length != other.bits.data.length || + numHashFunctions != other.getNumHashFunctions() || + hashing.type() != other.hashing.type()) { + throw new IllegalArgumentException("BloomFilters must have same size, number of hash functions, and hash type"); + } + + bits.or(other.bits); + } + @Override public int hashCode() { - return bits.hashCode() + numHashFunctions; + return bits.hashCode() + numHashFunctions + hashing.type(); } /* @@ -338,6 +358,18 @@ boolean set(long index) { return false; } + /** + * Union. BitArray's must be the same length. + */ + void or(BitArray other) { + assert data.length == other.data.length : "BitArrays must be same length"; + bitCount = 0; + for (int i = 0; i < data.length; i++) { + data[i] |= other.data[i]; + bitCount += Long.bitCount(data[i]); + } + } + boolean get(long index) { return (data[(int) (index >>> 6)] & (1L << index)) != 0; } diff --git a/src/main/java/org/elasticsearch/common/util/BytesRefHash.java b/src/main/java/org/elasticsearch/common/util/BytesRefHash.java index 90ee3c75f7112..61da83bfd5127 100644 --- a/src/main/java/org/elasticsearch/common/util/BytesRefHash.java +++ b/src/main/java/org/elasticsearch/common/util/BytesRefHash.java @@ -71,9 +71,9 @@ public BytesRef get(long id, BytesRef dest) { } /** - * Get the id associated with key + * Get the id associated with key using the provided spare BytesRef */ - public long find(BytesRef key, int code) { + public long find(BytesRef key, int code, BytesRef spare) { final long slot = slot(rehash(code), mask); for (long index = slot; ; index = nextSlot(index, mask)) { final long id = id(index); @@ -83,6 +83,13 @@ public long find(BytesRef key, int code) { } } + /** + * Get the id associated with key + */ + public long find(BytesRef key, int code) { + return find(key, code, spare); + } + /** Sugar for {@link #find(BytesRef, int) find(key, key.hashCode()} */ public long find(BytesRef key) { return find(key, key.hashCode()); @@ -151,6 +158,13 @@ public long add(BytesRef key) { return add(key, key.hashCode()); } + + /** Get the cached hashcode **/ + public int code(long id) { + assert id >= 0; + return hashes.get(id); + } + @Override protected void removeAndAdd(long index) { final long id = id(index, -1); diff --git a/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java b/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java index 4099eedd13e05..a56b962d3d7f4 100644 --- a/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java +++ b/src/main/java/org/elasticsearch/index/query/TermsFilterParser.java @@ -26,17 +26,22 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.Filter; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.terms.TermsByQueryRequest; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.lucene.search.AndFilter; import org.elasticsearch.common.lucene.search.OrFilter; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.XBooleanFilter; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.cache.filter.support.CacheKeyFilter; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.indices.cache.filter.terms.FieldTermsLookup; import org.elasticsearch.indices.cache.filter.terms.IndicesTermsFilterCache; +import org.elasticsearch.indices.cache.filter.terms.QueryTermsLookup; import org.elasticsearch.indices.cache.filter.terms.TermsLookup; import java.io.IOException; @@ -75,11 +80,17 @@ public Filter parse(QueryParseContext parseContext) throws IOException, QueryPar String filterName = null; String currentFieldName = null; - String lookupIndex = parseContext.index().name(); - String lookupType = null; String lookupId = null; String lookupPath = null; String lookupRouting = null; + List lookupIndices = Lists.newArrayList(); + List lookupTypes = Lists.newArrayList(); + XContentBuilder lookupFilter = null; + boolean lookupUseBloomFilter = false; + Double lookupBloomFpp = null; + Integer lookupBloomExpectedInsertions = null; + Integer lookupBloomHashFunctions = null; + Long lookupMaxTermsPerShard = null; boolean lookupCache = true; CacheKeyFilter.Key cacheKey = null; @@ -105,15 +116,70 @@ public Filter parse(QueryParseContext parseContext) throws IOException, QueryPar while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); + } else if (token == XContentParser.Token.START_ARRAY) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + String value = parser.text(); + if ("indices".equals(currentFieldName) || "index".equals(currentFieldName)) { + if (value != null) { + lookupIndices.add(value); + } + } else if ("types".equals(currentFieldName) || "type".equals(currentFieldName)) { + if (value != null) { + lookupTypes.add(value); + } + } + } + } else if (token == XContentParser.Token.START_OBJECT) { + if ("filter".equals(currentFieldName)) { + lookupFilter = XContentFactory.contentBuilder(parser.contentType()); + lookupFilter.copyCurrentStructure(parser); + } else if ("bloom_filter".equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + lookupUseBloomFilter = true; + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else if (token.isValue()) { + if ("fpp".equals(currentFieldName) || + "false_positive_probability".equals(currentFieldName) || + "falsePositiveProbability".equals(currentFieldName)) { + lookupBloomFpp = parser.doubleValue(); + if (lookupBloomFpp <= 0 || lookupBloomFpp >= 1) { + throw new QueryParsingException(parseContext.index(), "bloom fpp must be between 0 and 1"); + } + } else if ("expected_insertions".equals(currentFieldName) || + "expectedInsertions".equals(currentFieldName)) { + lookupBloomExpectedInsertions = parser.intValue(); + if (lookupBloomExpectedInsertions <= 0) { + throw new QueryParsingException(parseContext.index(), "bloom expected insertions greater than 0"); + } + } else if ("hash_functions".equals(currentFieldName) || + "hashFunctions".equals(currentFieldName)) { + lookupBloomHashFunctions = parser.intValue(); + if (lookupBloomHashFunctions < 1 || lookupBloomHashFunctions > 255) { + throw new QueryParsingException(parseContext.index(), "bloom hash functions must be between 1 and 255"); + } + } else { + throw new QueryParsingException(parseContext.index(), + "[terms] filter does not support [" + currentFieldName + "] within bloom element"); + } + } + } + } else { + throw new QueryParsingException(parseContext.index(), "[terms] filter does not support [" + currentFieldName + "] within lookup element"); + } } else if (token.isValue()) { - if ("index".equals(currentFieldName)) { - lookupIndex = parser.text(); - } else if ("type".equals(currentFieldName)) { - lookupType = parser.text(); + if ("index".equals(currentFieldName) || "indices".equals(currentFieldName)) { + lookupIndices.clear(); + lookupIndices.add(parser.text()); + } else if ("type".equals(currentFieldName) || "types".equals(currentFieldName)) { + lookupTypes.clear(); + lookupTypes.add(parser.text()); } else if ("id".equals(currentFieldName)) { lookupId = parser.text(); } else if ("path".equals(currentFieldName)) { lookupPath = parser.text(); + } else if ("max_terms_per_shard".equals(currentFieldName) || "maxTermsPerShard".equals(currentFieldName)) { + lookupMaxTermsPerShard = parser.longValue(); } else if ("routing".equals(currentFieldName)) { lookupRouting = parser.textOrNull(); } else if ("cache".equals(currentFieldName)) { @@ -123,15 +189,29 @@ public Filter parse(QueryParseContext parseContext) throws IOException, QueryPar } } } - if (lookupType == null) { - throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the type"); - } - if (lookupId == null) { - throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the id"); + + if (lookupFilter == null) { + if (lookupIndices.size() == 0) { + lookupIndices.add(parseContext.index().name()); + } + + if (lookupTypes == null || lookupTypes.size() == 0) { + throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the type"); + } + + if (lookupId == null) { + throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the id"); + } } + if (lookupPath == null) { throw new QueryParsingException(parseContext.index(), "[terms] filter lookup element requires specifying the path"); } + + if (lookupUseBloomFilter && lookupBloomExpectedInsertions == null) { + throw new QueryParsingException(parseContext.index(), + "[terms] filter lookup with bloom filter requires the expected number of insertions"); + } } else if (token.isValue()) { if ("execution".equals(currentFieldName)) { execution = parser.text(); @@ -148,7 +228,7 @@ public Filter parse(QueryParseContext parseContext) throws IOException, QueryPar } if (fieldName == null) { - throw new QueryParsingException(parseContext.index(), "terms filter requires a field name, followed by array of terms"); + throw new QueryParsingException(parseContext.index(), "terms filter requires a field name"); } FieldMapper fieldMapper = null; @@ -165,7 +245,7 @@ public Filter parse(QueryParseContext parseContext) throws IOException, QueryPar } } - if (lookupId != null) { + if (lookupId != null || lookupFilter != null) { // if there are no mappings, then nothing has been indexing yet against this shard, so we can return // no match (but not cached!), since the Terms Lookup relies on the fact that there are mappings... if (fieldMapper == null) { @@ -173,7 +253,41 @@ public Filter parse(QueryParseContext parseContext) throws IOException, QueryPar } // external lookup, use it - TermsLookup termsLookup = new TermsLookup(fieldMapper, lookupIndex, lookupType, lookupId, lookupRouting, lookupPath, parseContext); + TermsLookup termsLookup; + if (lookupFilter != null) { + final TermsByQueryRequest termsByQueryReq = new TermsByQueryRequest(lookupIndices.toArray(new String[lookupIndices.size()])) + .types(lookupTypes.toArray(new String[lookupTypes.size()])) + .field(lookupPath) + .routing(lookupRouting) + .filter(lookupFilter) + .useBloomFilter(lookupUseBloomFilter); + + if (lookupMaxTermsPerShard != null) { + termsByQueryReq.maxTermsPerShard(lookupMaxTermsPerShard); + } + + if (lookupUseBloomFilter && lookupBloomFpp != null) { + termsByQueryReq.bloomFpp(lookupBloomFpp); + } + + if (lookupUseBloomFilter && lookupBloomExpectedInsertions != null) { + termsByQueryReq.bloomExpectedInsertions(lookupBloomExpectedInsertions); + } + + if (lookupUseBloomFilter && lookupBloomHashFunctions != null) { + termsByQueryReq.bloomHashFunctions(lookupBloomHashFunctions); + } + + // default to no caching for query terms lookup + if (cache == null) { + cache = false; + } + + termsLookup = new QueryTermsLookup(termsByQueryReq, parseContext.fieldData().getForField(fieldMapper)); + } else { + termsLookup = new FieldTermsLookup(fieldMapper, lookupIndices.get(0), lookupTypes.get(0), + lookupId, lookupRouting, lookupPath, parseContext); + } Filter filter = termsFilterCache.termsFilter(termsLookup, lookupCache, cacheKey); if (filter == null) { @@ -184,6 +298,7 @@ public Filter parse(QueryParseContext parseContext) throws IOException, QueryPar if (cache == null || cache) { filter = parseContext.cacheFilter(filter, cacheKey); } + return filter; } diff --git a/src/main/java/org/elasticsearch/index/query/TermsLookupFilterBuilder.java b/src/main/java/org/elasticsearch/index/query/TermsLookupFilterBuilder.java index 1c23c8f338ce2..f6b28af14030a 100644 --- a/src/main/java/org/elasticsearch/index/query/TermsLookupFilterBuilder.java +++ b/src/main/java/org/elasticsearch/index/query/TermsLookupFilterBuilder.java @@ -24,21 +24,25 @@ import java.io.IOException; /** - * A filer for a field based on several terms matching on any of them. + * A filter for a field based on several terms matching on any of them. */ public class TermsLookupFilterBuilder extends BaseFilterBuilder { private final String name; - private String lookupIndex; - private String lookupType; - private String lookupId; - private String lookupRouting; - private String lookupPath; + private String[] indices; + private String[] types; + private String id; + private String routing; + private String path; private Boolean lookupCache; - + private FilterBuilder lookupFilter; private Boolean cache; private String cacheKey; private String filterName; + private Double bloomFpp; + private Integer bloomExpectedInsertions; + private Integer bloomHashFunctions; + private Long maxTermsPerShard; public TermsLookupFilterBuilder(String name) { this.name = name; @@ -55,50 +59,118 @@ public TermsLookupFilterBuilder filterName(String filterName) { /** * Sets the index name to lookup the terms from. */ - public TermsLookupFilterBuilder lookupIndex(String lookupIndex) { - this.lookupIndex = lookupIndex; + public TermsLookupFilterBuilder index(String index) { + this.indices = new String[]{index}; + return this; + } + + /** + * Sets the index name to lookup the terms from. + */ + public TermsLookupFilterBuilder indices(String... indices) { + this.indices = indices; + return this; + } + + /** + * Sets the index type to lookup the terms from. + */ + public TermsLookupFilterBuilder type(String type) { + this.types = new String[]{type}; return this; } /** * Sets the index type to lookup the terms from. */ - public TermsLookupFilterBuilder lookupType(String lookupType) { - this.lookupType = lookupType; + public TermsLookupFilterBuilder types(String... types) { + this.types = types; return this; } /** * Sets the doc id to lookup the terms from. */ - public TermsLookupFilterBuilder lookupId(String lookupId) { - this.lookupId = lookupId; + public TermsLookupFilterBuilder id(String id) { + this.id = id; return this; } /** * Sets the path within the document to lookup the terms from. */ - public TermsLookupFilterBuilder lookupPath(String lookupPath) { - this.lookupPath = lookupPath; + public TermsLookupFilterBuilder path(String path) { + this.path = path; + return this; + } + + /** + * Sets the filter used to lookup terms with + */ + public TermsLookupFilterBuilder lookupFilter(FilterBuilder lookupFilter) { + this.lookupFilter = lookupFilter; return this; } - public TermsLookupFilterBuilder lookupRouting(String lookupRouting) { - this.lookupRouting = lookupRouting; + /** + * Sets the node routing used to control the shards the lookup request is executed on + */ + public TermsLookupFilterBuilder routing(String routing) { + this.routing = routing; + return this; + } + + /** + * Sets the BloomFilter false positive probability + */ + public TermsLookupFilterBuilder bloomFpp(double bloomFpp) { + this.bloomFpp = bloomFpp; + return this; + } + + /** + * Sets the expected insertions to create the BloomFilter with + */ + public TermsLookupFilterBuilder bloomExpectedInsertions(int bloomExpectedInsertions) { + this.bloomExpectedInsertions = bloomExpectedInsertions; + return this; + } + + /** + * Sets the number of hash functions used in the BloomFilter + */ + public TermsLookupFilterBuilder bloomHashFunctions(int bloomHashFunctions) { + this.bloomHashFunctions = bloomHashFunctions; return this; } + /** + * Sets the max number of terms to collect per shard. + */ + public TermsLookupFilterBuilder maxTermsPerShard(long maxTermsPerShard) { + this.maxTermsPerShard = maxTermsPerShard; + return this; + } + + /** + * Sets if the gathered terms should be cached or not + */ public TermsLookupFilterBuilder lookupCache(boolean lookupCache) { this.lookupCache = lookupCache; return this; } + /** + * Sets if the resulting filter should be cached or not + */ public TermsLookupFilterBuilder cache(boolean cache) { this.cache = cache; return this; } + /** + * Sets the filter cache key + */ public TermsLookupFilterBuilder cacheKey(String cacheKey) { this.cacheKey = cacheKey; return this; @@ -107,32 +179,70 @@ public TermsLookupFilterBuilder cacheKey(String cacheKey) { @Override public void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(TermsFilterParser.NAME); - builder.startObject(name); - if (lookupIndex != null) { - builder.field("index", lookupIndex); + if (indices != null) { + builder.field("indices", indices); + } + + if (types != null) { + builder.field("types", types); + } + + if (id != null) { + builder.field("id", id); } - builder.field("type", lookupType); - builder.field("id", lookupId); - if (lookupRouting != null) { - builder.field("routing", lookupRouting); + + if (routing != null) { + builder.field("routing", routing); } + if (lookupCache != null) { builder.field("cache", lookupCache); } - builder.field("path", lookupPath); + + if (lookupFilter != null) { + builder.field("filter", lookupFilter); + } + + if (path != null) { + builder.field("path", path); + } + + if (maxTermsPerShard != null) { + builder.field("max_terms_per_shard", maxTermsPerShard); + } + + if (bloomFpp != null || bloomExpectedInsertions != null || bloomHashFunctions != null) { + builder.startObject("bloom_filter"); + + if (bloomFpp != null) { + builder.field("fpp", bloomFpp); + } + + if (bloomExpectedInsertions != null) { + builder.field("expected_insertions", bloomExpectedInsertions); + } + + if (bloomHashFunctions != null) { + builder.field("hash_functions", bloomHashFunctions); + } + + builder.endObject(); + } + builder.endObject(); if (filterName != null) { builder.field("_name", filterName); } + if (cache != null) { builder.field("_cache", cache); } + if (cacheKey != null) { builder.field("_cache_key", cacheKey); } - builder.endObject(); } -} \ No newline at end of file +} diff --git a/src/main/java/org/elasticsearch/index/search/BloomFieldDataTermsFilter.java b/src/main/java/org/elasticsearch/index/search/BloomFieldDataTermsFilter.java new file mode 100644 index 0000000000000..f5f19907d8b00 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/search/BloomFieldDataTermsFilter.java @@ -0,0 +1,105 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.search; + +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.lucene.docset.MatchDocIdSet; +import org.elasticsearch.common.util.BloomFilter; +import org.elasticsearch.index.fielddata.BytesValues; +import org.elasticsearch.index.fielddata.IndexFieldData; + +import java.io.IOException; + +/** + * Filters on non-numeric field values that might be found in a {@link BloomFilter}. + */ +public class BloomFieldDataTermsFilter extends FieldDataTermsFilter { + + final BloomFilter bloomFilter; + Integer hashCode; + + /** + * Constructor accepting a {@link BloomFilter}. + */ + public BloomFieldDataTermsFilter(IndexFieldData fieldData, BloomFilter bloomFilter) { + super(fieldData); + this.bloomFilter = bloomFilter; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || !(obj instanceof BloomFieldDataTermsFilter)) return false; + + BloomFieldDataTermsFilter that = (BloomFieldDataTermsFilter) obj; + if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; + if (!bloomFilter.equals(that.bloomFilter)) return false; + + return true; + } + + @Override + public int hashCode() { + if (hashCode == null) { + // calculate hashCode of each bloom filter + int hash = fieldData.getFieldNames().indexName().hashCode(); + hash += bloomFilter != null ? bloomFilter.hashCode() : 0; + + hashCode = hash; + } + + return hashCode; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("BloomFieldDataTermsFilter:"); + return sb + .append(fieldData.getFieldNames().indexName()) + .append(":") + .append(bloomFilter.toString()) // TODO: what to use here? hashCode maybe? + .toString(); + } + + @Override + public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException { + // make sure there are terms to filter on + if (bloomFilter == null || bloomFilter.isEmpty()) return null; + + final BytesValues values = fieldData.load(context).getBytesValues(false); // load fielddata + return new MatchDocIdSet(context.reader().maxDoc(), acceptDocs) { + @Override + protected boolean matchDoc(int doc) { + final int numVals = values.setDocument(doc); + for (int i = 0; i < numVals; i++) { + final BytesRef term = values.nextValue(); + if (bloomFilter.mightContain(term)) { + return true; + } + } + + return false; + } + }; + } +} diff --git a/src/main/java/org/elasticsearch/index/search/FieldDataTermsFilter.java b/src/main/java/org/elasticsearch/index/search/FieldDataTermsFilter.java index 86bf353a71a0a..690f26aa59203 100644 --- a/src/main/java/org/elasticsearch/index/search/FieldDataTermsFilter.java +++ b/src/main/java/org/elasticsearch/index/search/FieldDataTermsFilter.java @@ -21,12 +21,15 @@ import com.carrotsearch.hppc.DoubleOpenHashSet; import com.carrotsearch.hppc.LongOpenHashSet; import com.carrotsearch.hppc.ObjectOpenHashSet; +import com.google.common.primitives.Longs; import org.apache.lucene.index.AtomicReaderContext; import org.apache.lucene.search.DocIdSet; import org.apache.lucene.search.Filter; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.elasticsearch.common.lucene.docset.MatchDocIdSet; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.common.util.LongHash; import org.elasticsearch.index.fielddata.*; import java.io.IOException; @@ -54,6 +57,18 @@ public static FieldDataTermsFilter newBytes(IndexFieldData fieldData, ObjectOpen return new BytesFieldDataFilter(fieldData, terms); } + /** + * Get a {@link FieldDataTermsFilter} that filters on non-numeric terms found in a + * {@link org.elasticsearch.common.util.BytesRefHash} + * + * @param fieldData The fielddata for the field. + * @param terms A {@link org.elasticsearch.common.util.BytesRefHash} of terms. + * @return the filter. + */ + public static FieldDataTermsFilter newBytes(IndexFieldData fieldData, BytesRefHash terms) { + return new HashedBytesFieldDataFilter(fieldData, terms); + } + /** * Get a {@link FieldDataTermsFilter} that filters on non-floating point numeric terms found in a hppc * {@link LongOpenHashSet}. @@ -66,6 +81,18 @@ public static FieldDataTermsFilter newLongs(IndexNumericFieldData fieldData, Lon return new LongsFieldDataFilter(fieldData, terms); } + /** + * Get a {@link FieldDataTermsFilter} that filters on non-floating point numeric terms found in a + * {@link org.elasticsearch.common.util.LongHash}. + * + * @param fieldData The fielddata for the field. + * @param terms A {@link org.elasticsearch.common.util.LongHash} of terms. + * @return the filter. + */ + public static FieldDataTermsFilter newLongs(IndexNumericFieldData fieldData, LongHash terms) { + return new HashedLongsFieldDataFilter(fieldData, terms); + } + /** * Get a {@link FieldDataTermsFilter} that filters on floating point numeric terms found in a hppc * {@link DoubleOpenHashSet}. @@ -78,17 +105,22 @@ public static FieldDataTermsFilter newDoubles(IndexNumericFieldData fieldData, D return new DoublesFieldDataFilter(fieldData, terms); } - @Override - public boolean equals(Object obj) { - if (this == obj) return true; - if (obj == null || !(obj instanceof FieldDataTermsFilter)) return false; - - FieldDataTermsFilter that = (FieldDataTermsFilter) obj; - if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; - if (this.hashCode() != obj.hashCode()) return false; - return true; + /** + * Get a {@link FieldDataTermsFilter} that filters on floating point numeric terms found in a + * {@link org.elasticsearch.common.util.LongHash}. Terms must be represented as long bits, ie. using + * Double.doubleToLongBits. + * + * @param fieldData The fielddata for the field. + * @param terms A {@link org.elasticsearch.common.util.LongHash} of terms. + * @return the filter. + */ + public static FieldDataTermsFilter newDoubles(IndexNumericFieldData fieldData, LongHash terms) { + return new HashedDoublesFieldDataFilter(fieldData, terms); } + @Override + public abstract boolean equals(Object obj); + @Override public abstract int hashCode(); @@ -101,17 +133,32 @@ public boolean equals(Object obj) { protected static class BytesFieldDataFilter extends FieldDataTermsFilter { final ObjectOpenHashSet terms; + Integer hashCode; protected BytesFieldDataFilter(IndexFieldData fieldData, ObjectOpenHashSet terms) { super(fieldData); this.terms = terms; } + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || !(obj instanceof BytesFieldDataFilter)) return false; + + BytesFieldDataFilter that = (BytesFieldDataFilter) obj; + if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; + if (!terms.equals(that.terms)) return false; + + return true; + } + @Override public int hashCode() { - int hashcode = fieldData.getFieldNames().indexName().hashCode(); - hashcode += terms != null ? terms.hashCode() : 0; - return hashcode; + if (hashCode == null) { + hashCode = fieldData.getFieldNames().indexName().hashCode() + (terms != null ? terms.hashCode() : 0); + } + + return hashCode; } @Override @@ -146,23 +193,121 @@ protected boolean matchDoc(int doc) { } } + /** + * Filters on non-numeric fields. + */ + protected static class HashedBytesFieldDataFilter extends FieldDataTermsFilter { + + final BytesRefHash terms; + Integer hashCode; + + protected HashedBytesFieldDataFilter(IndexFieldData fieldData, BytesRefHash terms) { + super(fieldData); + this.terms = terms; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || !(obj instanceof HashedBytesFieldDataFilter)) return false; + + HashedBytesFieldDataFilter that = (HashedBytesFieldDataFilter) obj; + if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; + if (terms.size() != that.terms.size()) return false; + + // TODO: best way to do this? + BytesRef spare = new BytesRef(); + for (long i = 0; i < terms.size(); i++) { + terms.get(i, spare); + if (that.terms.find(spare, spare.hashCode()) < 0) return false; + } + + return true; + } + + @Override + public int hashCode() { + if (hashCode == null) { + hashCode = fieldData.getFieldNames().indexName().hashCode(); + if (terms != null) { + for (long i = 0; i < terms.size(); i++) { + hashCode += terms.code(i); + } + } + } + + return hashCode; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HashedBytesFieldDataFilter:"); + return sb + .append(fieldData.getFieldNames().indexName()) + .append(":") + .append(terms != null ? terms.toString() : "") // TODO: do better? + .toString(); + } + + @Override + public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException { + // make sure there are terms to filter on + if (terms == null || terms.size() == 0) return null; + + final BytesValues values = fieldData.load(context).getBytesValues(true); // load fielddata + return new MatchDocIdSet(context.reader().maxDoc(), acceptDocs) { + private final BytesRef spare = new BytesRef(); + + @Override + protected boolean matchDoc(int doc) { + + final int numVals = values.setDocument(doc); + for (int i = 0; i < numVals; i++) { + BytesRef term = values.nextValue(); + if (terms.find(term, values.currentValueHash(), spare) >= 0) { + return true; + } + } + + return false; + + } + }; + } + } + /** * Filters on non-floating point numeric fields. */ protected static class LongsFieldDataFilter extends FieldDataTermsFilter { final LongOpenHashSet terms; + Integer hashCode; protected LongsFieldDataFilter(IndexNumericFieldData fieldData, LongOpenHashSet terms) { super(fieldData); this.terms = terms; } + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || !(obj instanceof LongsFieldDataFilter)) return false; + + LongsFieldDataFilter that = (LongsFieldDataFilter) obj; + if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; + if (!terms.equals(that.terms)) return false; + + return true; + } + @Override public int hashCode() { - int hashcode = fieldData.getFieldNames().indexName().hashCode(); - hashcode += terms != null ? terms.hashCode() : 0; - return hashcode; + if (hashCode == null) { + hashCode = fieldData.getFieldNames().indexName().hashCode() + (terms != null ? terms.hashCode() : 0); + } + + return hashCode; } @Override @@ -204,23 +349,122 @@ protected boolean matchDoc(int doc) { } } + /** + * Filters on non-floating point numeric fields. + */ + protected static class HashedLongsFieldDataFilter extends FieldDataTermsFilter { + + final LongHash terms; + Integer hashCode; + + protected HashedLongsFieldDataFilter(IndexNumericFieldData fieldData, LongHash terms) { + super(fieldData); + this.terms = terms; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || !(obj instanceof HashedLongsFieldDataFilter)) return false; + + HashedLongsFieldDataFilter that = (HashedLongsFieldDataFilter) obj; + if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; + if (!terms.equals(that.terms)) return false; + if (terms.size() != that.terms.size()) return false; + + // TODO: best way to do this? + for (long i = 0; i < terms.size(); i++) { + if (that.terms.find(i) < 0) return false; + } + + return true; + } + + @Override + public int hashCode() { + if (hashCode == null) { + hashCode = fieldData.getFieldNames().indexName().hashCode(); + if (terms != null) { + for (long i = 0; i < terms.size(); i++) { + hashCode += Longs.hashCode(LongHash.hash(terms.get(i))); + } + } + } + + return hashCode; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HashedLongsFieldDataFilter:"); + return sb + .append(fieldData.getFieldNames().indexName()) + .append(":") + .append(terms != null ? terms.toString() : "") // TODO: do better + .toString(); + } + + @Override + public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException { + // make sure there are terms to filter on + if (terms == null || terms.size() == 0) return null; + + IndexNumericFieldData numericFieldData = (IndexNumericFieldData) fieldData; + if (!numericFieldData.getNumericType().isFloatingPoint()) { + final LongValues values = numericFieldData.load(context).getLongValues(); // load fielddata + return new MatchDocIdSet(context.reader().maxDoc(), acceptDocs) { + @Override + protected boolean matchDoc(int doc) { + final int numVals = values.setDocument(doc); + for (int i = 0; i < numVals; i++) { + if (terms.find(values.nextValue()) >= 0) { + return true; + } + } + + return false; + } + }; + } + + // only get here if wrong fielddata type in which case + // no docs will match so we just return null. + return null; + } + } + /** * Filters on floating point numeric fields. */ protected static class DoublesFieldDataFilter extends FieldDataTermsFilter { final DoubleOpenHashSet terms; + Integer hashCode; protected DoublesFieldDataFilter(IndexNumericFieldData fieldData, DoubleOpenHashSet terms) { super(fieldData); this.terms = terms; } + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || !(obj instanceof DoublesFieldDataFilter)) return false; + + DoublesFieldDataFilter that = (DoublesFieldDataFilter) obj; + if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; + if (!terms.equals(that.terms)) return false; + + return true; + } + @Override public int hashCode() { - int hashcode = fieldData.getFieldNames().indexName().hashCode(); - hashcode += terms != null ? terms.hashCode() : 0; - return hashcode; + if (hashCode == null) { + hashCode = fieldData.getFieldNames().indexName().hashCode() + (terms != null ? terms.hashCode() : 0); + } + + return hashCode; } @Override @@ -263,4 +507,90 @@ protected boolean matchDoc(int doc) { return null; } } + + /** + * Filters on floating point numeric fields. + */ + protected static class HashedDoublesFieldDataFilter extends FieldDataTermsFilter { + + final LongHash terms; + Integer hashCode; + + protected HashedDoublesFieldDataFilter(IndexNumericFieldData fieldData, LongHash terms) { + super(fieldData); + this.terms = terms; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) return true; + if (obj == null || !(obj instanceof HashedLongsFieldDataFilter)) return false; + + HashedLongsFieldDataFilter that = (HashedLongsFieldDataFilter) obj; + if (!fieldData.getFieldNames().indexName().equals(that.fieldData.getFieldNames().indexName())) return false; + if (!terms.equals(that.terms)) return false; + if (terms.size() != that.terms.size()) return false; + + // TODO: best way to do this? + for (long i = 0; i < terms.size(); i++) { + if (that.terms.find(i) < 0) return false; + } + + return true; + } + + @Override + public int hashCode() { + if (hashCode == null) { + hashCode = fieldData.getFieldNames().indexName().hashCode(); + if (terms != null) { + for (long i = 0; i < terms.size(); i++) { + hashCode += Longs.hashCode(LongHash.hash(terms.get(i))); + } + } + } + + return hashCode; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("HashedDoublesFieldDataFilter:"); + return sb + .append(fieldData.getFieldNames().indexName()) + .append(":") + .append(terms != null ? terms.toString() : "") // TODO: do better + .toString(); + } + + @Override + public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException { + // make sure there are terms to filter on + if (terms == null || terms.size() == 0) return null; + + IndexNumericFieldData numericFieldData = (IndexNumericFieldData) fieldData; + if (numericFieldData.getNumericType().isFloatingPoint()) { + final DoubleValues values = numericFieldData.load(context).getDoubleValues(); // load fielddata + return new MatchDocIdSet(context.reader().maxDoc(), acceptDocs) { + @Override + protected boolean matchDoc(int doc) { + final int numVals = values.setDocument(doc); + for (int i = 0; i < numVals; i++) { + final double dval = values.nextValue(); + final long lval = Double.doubleToLongBits(dval); + if (terms.find(lval) >= 0) { + return true; + } + } + + return false; + } + }; + } + + // only get here if wrong fielddata type in which case + // no docs will match so we just return null. + return null; + } + } } diff --git a/src/main/java/org/elasticsearch/indices/cache/filter/terms/FieldTermsLookup.java b/src/main/java/org/elasticsearch/indices/cache/filter/terms/FieldTermsLookup.java new file mode 100644 index 0000000000000..c97dd1989121a --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/cache/filter/terms/FieldTermsLookup.java @@ -0,0 +1,104 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.indices.cache.filter.terms; + +import org.apache.lucene.search.Filter; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.get.GetRequest; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.mapper.FieldMapper; +import org.elasticsearch.index.query.QueryParseContext; + +import java.util.List; + +/** + * A {@link TermsLookup} implementation that gathers the filter terms from the field of another document. + */ +public class FieldTermsLookup extends TermsLookup { + + private final FieldMapper fieldMapper; + private final String index; + private final String type; + private final String id; + private final String routing; + private final String path; + @Nullable + private final QueryParseContext queryParseContext; + private List terms; + + public FieldTermsLookup(FieldMapper fieldMapper, String index, String type, String id, String routing, String path, + @Nullable QueryParseContext queryParseContext) { + this.fieldMapper = fieldMapper; + this.index = index; + this.type = type; + this.id = id; + this.routing = routing; + this.path = path; + this.queryParseContext = queryParseContext; + } + + /** + * Performs a {@link GetRequest} for the document containing the lookup terms, extracts the terms from the specified + * field and generates an {@link org.apache.lucene.queries.TermsFilter}. + * @return the generated filter + */ + @Override + public Filter getFilter() { + GetResponse getResponse = client.get(new GetRequest(index, type, id).preference("_local").routing(routing)).actionGet(); + if (!getResponse.isExists()) { + return null; + } + + terms = XContentMapValues.extractRawValues(path, getResponse.getSourceAsMap()); + if (terms.isEmpty()) { + return null; + } + + Filter filter = fieldMapper.termsFilter(terms, queryParseContext); + return filter; + } + + /** + * Estimates the size of the filter. + * @return the estimated size in bytes. + */ + @Override + public long estimateSizeInBytes() { + long size = 8; + for (Object term : terms) { + if (term instanceof BytesRef) { + size += ((BytesRef) term).length; + } else if (term instanceof String) { + size += ((String) term).length() / 2; + } else { + size += 4; + } + } + + return size; + } + + @Override + public String toString() { + return fieldMapper.names().fullName() + ":" + index + "/" + type + "/" + id + "/" + path; + } +} diff --git a/src/main/java/org/elasticsearch/indices/cache/filter/terms/IndicesTermsFilterCache.java b/src/main/java/org/elasticsearch/indices/cache/filter/terms/IndicesTermsFilterCache.java index bad9a24a19b4f..e75d451a990f8 100644 --- a/src/main/java/org/elasticsearch/indices/cache/filter/terms/IndicesTermsFilterCache.java +++ b/src/main/java/org/elasticsearch/indices/cache/filter/terms/IndicesTermsFilterCache.java @@ -25,8 +25,6 @@ import org.apache.lucene.search.Filter; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.get.GetRequest; -import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.client.Client; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractComponent; @@ -36,10 +34,8 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.cache.filter.support.CacheKeyFilter; -import java.util.List; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -49,11 +45,14 @@ public class IndicesTermsFilterCache extends AbstractComponent { private static TermsFilterValue NO_TERMS = new TermsFilterValue(0, Queries.MATCH_NO_FILTER); - private final Client client; - private final Cache cache; + /** + * This cache will not actually cache any values. + */ + private final Cache shardSyncCache; + @Inject public IndicesTermsFilterCache(Settings settings, Client client) { super(settings); @@ -75,12 +74,28 @@ public IndicesTermsFilterCache(Settings settings, Client client) { } this.cache = builder.build(); + this.shardSyncCache = CacheBuilder.newBuilder().maximumSize(0).build(); } @Nullable public Filter termsFilter(final TermsLookup lookup, boolean cacheLookup, @Nullable CacheKeyFilter.Key cacheKey) throws RuntimeException { + // TODO: figure out how to inject client into abstract terms lookup + lookup.setClient(client); + + final Cache lookupCache; if (!cacheLookup) { - return buildTermsFilterValue(lookup).filter; + /* + Use the shardSyncCache which never actually caches a response. The reason we use this is to prevent + duplicate lookup requests (ie. from multiple shards on the same machine). This works because a cache will + block threads requesting a cache value that is already being loaded. So the first shard requests a lookup + value which triggers TermsLookup#getFilter which is responsible for doing the heavy term gathering + via GetRequest, Query, etc. The other shards will request the same lookup value and the cache will + block those requests until the original request is finished and then send response to all threads + waiting for the same lookup value. + */ + lookupCache = shardSyncCache; + } else { + lookupCache = cache; } BytesRef key; @@ -89,8 +104,9 @@ public Filter termsFilter(final TermsLookup lookup, boolean cacheLookup, @Nullab } else { key = new BytesRef(lookup.toString()); } + try { - return cache.get(key, new Callable() { + return lookupCache.get(key, new Callable() { @Override public TermsFilterValue call() throws Exception { return buildTermsFilterValue(lookup); @@ -105,30 +121,12 @@ public TermsFilterValue call() throws Exception { } TermsFilterValue buildTermsFilterValue(TermsLookup lookup) { - GetResponse getResponse = client.get(new GetRequest(lookup.getIndex(), lookup.getType(), lookup.getId()).preference("_local").routing(lookup.getRouting())).actionGet(); - if (!getResponse.isExists()) { + Filter filter = lookup.getFilter(); + if (filter == null) { return NO_TERMS; } - List values = XContentMapValues.extractRawValues(lookup.getPath(), getResponse.getSourceAsMap()); - if (values.isEmpty()) { - return NO_TERMS; - } - Filter filter = lookup.getFieldMapper().termsFilter(values, lookup.getQueryParseContext()); - return new TermsFilterValue(estimateSizeInBytes(values), filter); - } - long estimateSizeInBytes(List terms) { - long size = 8; - for (Object term : terms) { - if (term instanceof BytesRef) { - size += ((BytesRef) term).length; - } else if (term instanceof String) { - size += ((String) term).length() / 2; - } else { - size += 4; - } - } - return size; + return new TermsFilterValue(lookup.estimateSizeInBytes(), filter); } public void clear(String reason) { diff --git a/src/main/java/org/elasticsearch/indices/cache/filter/terms/QueryTermsLookup.java b/src/main/java/org/elasticsearch/indices/cache/filter/terms/QueryTermsLookup.java new file mode 100644 index 0000000000000..eaf46eb36fb1f --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/cache/filter/terms/QueryTermsLookup.java @@ -0,0 +1,124 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.indices.cache.filter.terms; + +import com.google.common.base.Joiner; +import org.apache.lucene.search.Filter; +import org.elasticsearch.action.terms.ResponseTerms; +import org.elasticsearch.action.terms.TermsByQueryAction; +import org.elasticsearch.action.terms.TermsByQueryRequest; +import org.elasticsearch.action.terms.TermsByQueryResponse; +import org.elasticsearch.common.util.BloomFilter; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.IndexNumericFieldData; +import org.elasticsearch.index.search.BloomFieldDataTermsFilter; +import org.elasticsearch.index.search.FieldDataTermsFilter; + +/** + * A {@link TermsLookup} implementation that gathers the filter terms from the specified field of documents matching + * the specified filter. + */ +public class QueryTermsLookup extends TermsLookup { + + private final TermsByQueryRequest request; + private final IndexFieldData fieldData; + private ResponseTerms terms; + + public QueryTermsLookup(TermsByQueryRequest request, IndexFieldData fieldData) { + super(); + this.request = request; + this.fieldData = fieldData; + } + + /** + * Executes the lookup query and gathers the terms. It generates a {@link FieldDataTermsFilter} that uses the + * fielddata to lookup and filter values. + * + * @return the lookup filter + */ + @Override + public Filter getFilter() { + TermsByQueryResponse termsByQueryResp = client.execute(TermsByQueryAction.INSTANCE, request).actionGet(); + terms = termsByQueryResp.getResponseTerms(); + + if (terms.size() == 0) { + return null; + } + + Filter filter; + switch (terms.getType()) { + case BLOOM: + filter = new BloomFieldDataTermsFilter(fieldData, (BloomFilter) terms.getTerms()); + break; + case LONGS: + filter = FieldDataTermsFilter.newLongs((IndexNumericFieldData) fieldData, (LongHash) terms.getTerms()); + break; + case DOUBLES: + filter = FieldDataTermsFilter.newDoubles((IndexNumericFieldData) fieldData, (LongHash) terms.getTerms()); + break; + default: + filter = FieldDataTermsFilter.newBytes(fieldData, (BytesRefHash) terms.getTerms()); + } + + + return filter; + } + + /** + * String representation of the query lookup. + * + * @return the string representation + */ + @Override + public String toString() { + // to_field/index1,index2,.../type1,type2,.../from_field/filter(filter_bytes_hash) + Joiner joiner = Joiner.on(","); + StringBuilder repr = new StringBuilder(fieldData.getFieldNames().indexName()) + .append(":").append(joiner.join(request.indices())).append("/") + .append(joiner.join(request.types())).append("/").append(request.field()).append("/"); + + if (request.filterSource() != null) { + repr.append("filter("); + try { + repr.append(request.filterSource().toBytesArray().hashCode()); + } finally { + repr.append(")"); + } + } + + if (request.useBloomFilter()) { + repr.append("/bloom"); + } + + return repr.toString(); + } + + /** + * Deletages to {@link org.elasticsearch.action.terms.ResponseTerms#getSizeInBytes()} + * + * @return the estimated size of the filter in bytes. + */ + @Override + public long estimateSizeInBytes() { + return terms.getSizeInBytes(); + } +} diff --git a/src/main/java/org/elasticsearch/indices/cache/filter/terms/TermsLookup.java b/src/main/java/org/elasticsearch/indices/cache/filter/terms/TermsLookup.java index 0bc72ee0be0e8..e362bc4d2f380 100644 --- a/src/main/java/org/elasticsearch/indices/cache/filter/terms/TermsLookup.java +++ b/src/main/java/org/elasticsearch/indices/cache/filter/terms/TermsLookup.java @@ -19,65 +19,44 @@ package org.elasticsearch.indices.cache.filter.terms; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.index.mapper.FieldMapper; -import org.elasticsearch.index.query.QueryParseContext; +import org.apache.lucene.search.Filter; +import org.elasticsearch.client.Client; + +import java.util.Collection; +import java.util.Iterator; /** + * Abstract {@link TermsLookup}. */ -public class TermsLookup { - - private final FieldMapper fieldMapper; - - private final String index; - private final String type; - private final String id; - private final String routing; - private final String path; - - @Nullable - private final QueryParseContext queryParseContext; - - public TermsLookup(FieldMapper fieldMapper, String index, String type, String id, String routing, String path, @Nullable QueryParseContext queryParseContext) { - this.fieldMapper = fieldMapper; - this.index = index; - this.type = type; - this.id = id; - this.routing = routing; - this.path = path; - this.queryParseContext = queryParseContext; - } - - public FieldMapper getFieldMapper() { - return fieldMapper; - } - - public String getIndex() { - return index; - } - - public String getType() { - return type; - } - - public String getId() { - return id; - } - - public String getRouting() { - return this.routing; - } - - public String getPath() { - return path; - } - - @Nullable - public QueryParseContext getQueryParseContext() { - return queryParseContext; - } - - public String toString() { - return fieldMapper.names().fullName() + ":" + index + "/" + type + "/" + id + "/" + path; - } +public abstract class TermsLookup { + + protected Client client; + + // TODO: Can this be injected? + /** + * Sets the client + * @param client the {@link Client} + */ + public void setClient(Client client) { + this.client = client; + } + + /** + * Returns the lookup filter + * @return the filter + */ + public abstract Filter getFilter(); + + /** + * Used for cache key when not specified + * @return the lookup string representation + */ + public abstract String toString(); + + /** + * The size of the lookup in bytes to be used in + * cache size calculations + * @return the size of the lookup in bytes + */ + public abstract long estimateSizeInBytes(); } diff --git a/src/test/java/org/elasticsearch/action/terms/SimpleTermsByQueryActionTests.java b/src/test/java/org/elasticsearch/action/terms/SimpleTermsByQueryActionTests.java new file mode 100644 index 0000000000000..df67d7ab74c30 --- /dev/null +++ b/src/test/java/org/elasticsearch/action/terms/SimpleTermsByQueryActionTests.java @@ -0,0 +1,213 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.terms; + +import com.carrotsearch.hppc.DoubleOpenHashSet; +import com.carrotsearch.hppc.LongOpenHashSet; +import com.carrotsearch.hppc.ObjectOpenHashSet; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.util.BloomFilter; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import org.junit.Test; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.query.FilterBuilders.matchAllFilter; +import static org.elasticsearch.index.query.FilterBuilders.rangeFilter; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +public class SimpleTermsByQueryActionTests extends ElasticsearchIntegrationTest { + + /** + * Tests that the terms by query action returns the correct terms against string fields + */ + @Test + public void testTermsByQuery() throws Exception { + int numShards = randomIntBetween(1, 6); + int numDocs = randomIntBetween(100, 2000); + long maxTermsPerShard = randomIntBetween(1, numDocs / numShards - 1); + + logger.info("--> creating index [idx] shards [" + numShards + "]"); + client().admin().indices().prepareCreate("idx").setSettings("index.number_of_shards", numShards, "index.number_of_replicas", 0).execute().actionGet(); + ensureGreen(); + + logger.info("--> indexing [" + numDocs + "] docs"); + for (int i = 0; i < numDocs; i++) { + client().prepareIndex("idx", "type", "" + i) + .setSource(jsonBuilder().startObject() + .field("str", Integer.toString(i)).field("dbl", Double.valueOf(i)).field("int", i) + .endObject()).execute().actionGet(); + } + + client().admin().indices().prepareRefresh("idx").execute().actionGet(); + + logger.info("--> lookup terms in field [str]"); + TermsByQueryResponse resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("str") + .setFilter(matchAllFilter()).execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is((long) numDocs)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.BytesResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof BytesRefHash, is(true)); + BytesRefHash bTerms = (BytesRefHash) resp.getResponseTerms().getTerms(); + assertThat(bTerms.size(), is((long) numDocs)); + for (int i = 0; i < numDocs; i++) { + BytesRef spare = new BytesRef(Integer.toString(i)); + assertThat(bTerms.find(spare, spare.hashCode()) >= 0, is(true)); + } + + logger.info("--> lookup terms in field [str] with max terms per shard"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("str") + .setFilter(matchAllFilter()).setMaxTermsPerShard(maxTermsPerShard).execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is(numShards * maxTermsPerShard)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.BytesResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof BytesRefHash, is(true)); + bTerms = (BytesRefHash) resp.getResponseTerms().getTerms(); + assertThat(bTerms.size(), is(numShards * maxTermsPerShard)); + + logger.info("--> lookup terms in field [str] with BloomFilter"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("str").setUseBloomFilter(true) + .setExpectedInsertions(numDocs).setFilter(matchAllFilter()).execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is((long) numDocs)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.BloomResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof BloomFilter, is(true)); + BloomFilter bloomFilter = (BloomFilter) resp.getResponseTerms().getTerms(); + for (int i = 0; i < numDocs; i++) { + assertThat(bloomFilter.mightContain(new BytesRef(Integer.toString(i))), is(true)); + } + + logger.info("--> lookup terms in field [str] with BloomFilter and max terms per shard"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("str").setUseBloomFilter(true) + .setExpectedInsertions(numDocs).setFilter(matchAllFilter()).setMaxTermsPerShard(maxTermsPerShard) + .execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is(numShards * maxTermsPerShard)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.BloomResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof BloomFilter, is(true)); + + logger.info("--> lookup terms in field [int]"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("int") + .setFilter(matchAllFilter()).execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is((long) numDocs)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.LongsResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof LongHash, is(true)); // bloom doesn't store terms + LongHash lTerms = (LongHash) resp.getResponseTerms().getTerms(); + assertThat(lTerms.size(), is((long) numDocs)); + for (int i = 0; i < numDocs; i++) { + assertThat(lTerms.find(Long.valueOf(i)) >= 0, is(true)); + } + + logger.info("--> lookup terms in field [int] with max terms per shard"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("int") + .setFilter(matchAllFilter()).setMaxTermsPerShard(maxTermsPerShard).execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is(numShards * maxTermsPerShard)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.LongsResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof LongHash, is(true)); // bloom doesn't store terms + lTerms = (LongHash) resp.getResponseTerms().getTerms(); + assertThat((long) lTerms.size(), is(numShards * maxTermsPerShard)); + + logger.info("--> lookup terms in field [dbl]"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("dbl") + .setFilter(matchAllFilter()).execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is((long) numDocs)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.DoublesResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof LongHash, is(true)); + LongHash dTerms = (LongHash) resp.getResponseTerms().getTerms(); + assertThat(dTerms.size(), is((long) numDocs)); + for (int i = 0; i < numDocs; i++) { + assertThat(dTerms.find(Double.doubleToLongBits(Double.valueOf(i))) >= 0, is(true)); + } + int found = 0; + for (long i = 0; i < dTerms.capacity(); i++) { + final long id = dTerms.id(i); + if (id >= 0) { + long lval = dTerms.get(id); + double dval = Double.longBitsToDouble(lval); + assertThat(dval >= 0 && dval < numDocs, is(true)); + found++; + } + } + assertThat(found, equalTo(numDocs)); + + logger.info("--> lookup terms in field [dbl] with max terms per shard"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("dbl") + .setFilter(matchAllFilter()).setMaxTermsPerShard(maxTermsPerShard).execute().actionGet(); + + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is(numShards * maxTermsPerShard)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.DoublesResponseTerms, is(true)); + assertThat(resp.getResponseTerms().getTerms() instanceof LongHash, is(true)); + dTerms = (LongHash) resp.getResponseTerms().getTerms(); + assertThat((long) dTerms.size(), is(numShards * maxTermsPerShard)); + + logger.info("--> lookup in field [str] with no docs"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("str") + .setFilter(rangeFilter("int").gt(numDocs)).execute().actionGet(); + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is(0L)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.BytesResponseTerms, is(true)); + bTerms = (BytesRefHash) resp.getResponseTerms().getTerms(); + assertThat(bTerms.size(), is(0L)); + + logger.info("--> lookup in field [int] with no docs"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("int") + .setFilter(rangeFilter("int").gt(numDocs)).execute().actionGet(); + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is(0L)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.LongsResponseTerms, is(true)); + lTerms = (LongHash) resp.getResponseTerms().getTerms(); + assertThat(lTerms.size(), is(0L)); + + logger.info("--> lookup in field [dbl] with no docs"); + resp = new TermsByQueryRequestBuilder(client()).setIndices("idx").setField("dbl") + .setFilter(rangeFilter("int").gt(numDocs)).execute().actionGet(); + ElasticsearchAssertions.assertNoFailures(resp); + assertThat(resp.getResponseTerms(), notNullValue()); + assertThat(resp.getResponseTerms().size(), is(0L)); + assertThat(resp.getResponseTerms() instanceof ResponseTerms.DoublesResponseTerms, is(true)); + dTerms = (LongHash) resp.getResponseTerms().getTerms(); + assertThat(dTerms.size(), is(0L)); + } +} diff --git a/src/test/java/org/elasticsearch/benchmark/search/lookup/QueryTermsLookupBenchmark.java b/src/test/java/org/elasticsearch/benchmark/search/lookup/QueryTermsLookupBenchmark.java new file mode 100644 index 0000000000000..f1d115d645d39 --- /dev/null +++ b/src/test/java/org/elasticsearch/benchmark/search/lookup/QueryTermsLookupBenchmark.java @@ -0,0 +1,668 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.benchmark.search.lookup; + +import org.elasticsearch.action.ListenableActionFuture; +import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.Requests; +import org.elasticsearch.common.StopWatch; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.query.FilterBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.TermsLookupFilterBuilder; +import org.elasticsearch.node.Node; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Random; +import java.util.Set; + +import static org.elasticsearch.client.Requests.createIndexRequest; +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; +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.FilterBuilders.*; +import static org.elasticsearch.index.query.QueryBuilders.filteredQuery; +import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; +import static org.elasticsearch.node.NodeBuilder.nodeBuilder; +import static org.elasticsearch.search.facet.FacetBuilders.termsFacet; + +/** + * Perform the parent/child search tests using terms query lookup. + * This should work across multiple shards and not need a special mapping + */ +public class QueryTermsLookupBenchmark { + + // index settings + public static final int NUM_SHARDS = 3; + public static final int NUM_REPLICAS = 0; + public static final String PARENT_INDEX = "joinparent"; + public static final String PARENT_TYPE = "p"; + public static final String CHILD_INDEX = "joinchild"; + public static final String CHILD_TYPE = "c"; + // test settings + public static final int NUM_PARENTS = 1000000; + public static final int NUM_CHILDREN_PER_PARENT = 5; + public static final int BATCH_SIZE = 100; + public static final int NUM_QUERIES = 50; + private final Node node; + private final Client client; + private final Random random; + + QueryTermsLookupBenchmark() { + Settings settings = settingsBuilder() + .put("index.engine.robin.refreshInterval", "-1") + .put("gateway.type", "local") + .put(SETTING_NUMBER_OF_SHARDS, NUM_SHARDS) + .put(SETTING_NUMBER_OF_REPLICAS, NUM_REPLICAS) + .build(); + + this.node = nodeBuilder().settings(settingsBuilder().put(settings).put("name", "node1")).node(); + this.client = node.client(); + this.random = new Random(System.currentTimeMillis()); + } + + public static void main(String[] args) throws Exception { + + QueryTermsLookupBenchmark bench = new QueryTermsLookupBenchmark(); + bench.waitForGreen(); + bench.setupIndex(); + bench.memStatus(); + + // don't cache lookup + bench.benchHasChildSingleTerm(false); + bench.benchHasParentSingleTerm(false); + bench.benchHasParentMatchAll(false); + bench.benchHasChildMatchAll(false); + bench.benchHasParentMatchAllWithMaxTerms(false); + bench.benchHasParentRandomTerms(false); + + System.gc(); + bench.memStatus(); + bench.shutdown(); + } + + public void waitForGreen() { + client.admin().cluster().prepareHealth().setWaitForGreenStatus().setTimeout("10s").execute().actionGet(); + } + + public void shutdown() { + client.close(); + node.close(); + } + + public void log(String msg) { + System.out.println("--> " + msg); + } + + public void memStatus() { + NodeStats nodeStats = client.admin().cluster().prepareNodesStats().setJvm(true).setIndices(true).execute().actionGet().getNodes()[0]; + log("==== MEMORY ===="); + log("Committed heap size: " + nodeStats.getJvm().getMem().getHeapCommitted()); + log("Used heap size: " + nodeStats.getJvm().getMem().getHeapUsed()); + log("FieldData cache size: " + nodeStats.getIndices().getFieldData().getMemorySize()); + log(""); + } + + public XContentBuilder parentSource(int id, String nameValue) throws IOException { + return jsonBuilder().startObject().field("id", Integer.toString(id)).field("num", id).field("name", nameValue) + .field("dnum", Double.valueOf(id)).endObject(); + } + + public XContentBuilder childSource(String id, int parent, String tag) throws IOException { + return jsonBuilder().startObject().field("id", id).field("pid", Integer.toString(parent)).field("num", parent) + .field("tag", tag).field("dnum", Double.valueOf(parent)).endObject(); + } + + public void setupIndex() { + log("==== INDEX SETUP ===="); + try { + client.admin().indices().create(createIndexRequest(PARENT_INDEX)).actionGet(); + client.admin().indices().create(createIndexRequest(CHILD_INDEX)).actionGet(); + Thread.sleep(5000); + + StopWatch stopWatch = new StopWatch().start(); + + log("Indexing [" + NUM_PARENTS + "] parent documents into [" + PARENT_INDEX + "]"); + log("Indexing [" + (NUM_PARENTS * NUM_CHILDREN_PER_PARENT) + "] child documents into [" + CHILD_INDEX + "]"); + int ITERS = NUM_PARENTS / BATCH_SIZE; + int i = 1; + int counter = 0; + for (; i <= ITERS; i++) { + BulkRequestBuilder request = client.prepareBulk(); + for (int j = 0; j < BATCH_SIZE; j++) { + String parentId = Integer.toString(counter); + counter++; + request.add(Requests.indexRequest(PARENT_INDEX) + .type(PARENT_TYPE) + .id(parentId) + .source(parentSource(counter, "test" + counter))); + + for (int k = 0; k < NUM_CHILDREN_PER_PARENT; k++) { + String childId = parentId + "_" + k; + request.add(Requests.indexRequest(CHILD_INDEX) + .type(CHILD_TYPE) + .id(childId) + .source(childSource(childId, counter, "tag" + k))); + } + } + + BulkResponse response = request.execute().actionGet(); + if (response.hasFailures()) { + log("Index Failures..."); + } + + if (((i * BATCH_SIZE) % 10000) == 0) { + log("Indexed [" + (i * BATCH_SIZE) * (1 + NUM_CHILDREN_PER_PARENT) + "] took [" + stopWatch.stop().lastTaskTime() + "]"); + stopWatch.start(); + } + } + + log("Indexing took [" + stopWatch.totalTime() + "]"); + log("TPS [" + (((double) (NUM_PARENTS * (1 + NUM_CHILDREN_PER_PARENT))) / stopWatch.totalTime().secondsFrac()) + "]"); + } catch (Exception e) { + log("Indices exist, wait for green"); + waitForGreen(); + } + + client.admin().indices().prepareRefresh().execute().actionGet(); + log("Number of docs in index: " + client.prepareCount(PARENT_INDEX, CHILD_INDEX).setQuery(matchAllQuery()).execute().actionGet().getCount()); + log(""); + } + + public void warmFieldData(String parentField, String childField) { + ListenableActionFuture parentSearch = null; + ListenableActionFuture childSearch = null; + + if (parentField != null) { + parentSearch = client + .prepareSearch(PARENT_INDEX) + .setQuery(matchAllQuery()).addFacet(termsFacet("parentfield").field(parentField)).execute(); + } + + if (childField != null) { + childSearch = client + .prepareSearch(CHILD_INDEX) + .setQuery(matchAllQuery()).addFacet(termsFacet("childfield").field(childField)).execute(); + } + + if (parentSearch != null) parentSearch.actionGet(); + if (childSearch != null) childSearch.actionGet(); + } + + public long runQuery(String name, int testNum, String index, long expectedHits, QueryBuilder query) { + SearchResponse searchResponse = client + .prepareSearch(index) + .setQuery(query) + .execute().actionGet(); + + if (searchResponse.getFailedShards() > 0) { + log("Search Failures " + Arrays.toString(searchResponse.getShardFailures())); + } + + long hits = searchResponse.getHits().totalHits(); + if (hits != expectedHits) { + log("[" + name + "][#" + testNum + "] Hits Mismatch: expected [" + expectedHits + "], got [" + hits + "]"); + } + + return searchResponse.getTookInMillis(); + } + + /** + * Search for parent documents that have children containing a specified tag. + * Expect all parents returned since one child from each parent will match the lookup. + *

+ * Parent string field = "id" + * Parent long field = "num" + * Parent double field = "dnum" + * Child string field = "pid" + * Child long field = "num" + * Child double field = "dnum" + */ + public void benchHasChildSingleTerm(boolean cacheLookup) { + FilterBuilder lookupFilter; + QueryBuilder mainQuery = matchAllQuery(); + + TermsLookupFilterBuilder stringFilter = termsLookupFilter("id") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("pid") + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder bloomFilter = termsLookupFilter("id") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("pid") + .bloomExpectedInsertions(NUM_PARENTS) + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder longFilter = termsLookupFilter("dnum") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("dnum") + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder doubleFilter = termsLookupFilter("dnum") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("dnum") + .lookupCache(cacheLookup); + + long tookString = 0; + long tookLong = 0; + long tookDouble = 0; + long tookBloom = 0; + long expected = NUM_PARENTS; + warmFieldData("id", "pid"); // for string fields + warmFieldData("num", "num"); // for long fields + warmFieldData("dnum", "dnum"); // for double fields + + log("==== HAS CHILD SINGLE TERM (cache: " + cacheLookup + ") ===="); + for (int i = 0; i < NUM_QUERIES; i++) { + lookupFilter = termFilter("tag", "tag" + random.nextInt(NUM_CHILDREN_PER_PARENT)); + + stringFilter.lookupFilter(lookupFilter); + longFilter.lookupFilter(lookupFilter); + doubleFilter.lookupFilter(lookupFilter); + bloomFilter.lookupFilter(lookupFilter); + + tookString += runQuery("string", i, PARENT_INDEX, expected, filteredQuery(mainQuery, stringFilter)); + tookLong += runQuery("long", i, PARENT_INDEX, expected, filteredQuery(mainQuery, longFilter)); + tookDouble += runQuery("double", i, PARENT_INDEX, expected, filteredQuery(mainQuery, doubleFilter)); + tookBloom += runQuery("bloom", i, PARENT_INDEX, expected, filteredQuery(mainQuery, bloomFilter)); + } + + log("string: " + (tookString / NUM_QUERIES) + "ms avg"); + log("long : " + (tookLong / NUM_QUERIES) + "ms avg"); + log("double: " + (tookDouble / NUM_QUERIES) + "ms avg"); + log("bloom : " + (tookBloom / NUM_QUERIES) + "ms avg"); + log(""); + } + + /** + * Search for parent documents that have any child. + * Expect all parent documents returned. + *

+ * Parent string field = "id" + * Parent long field = "num" + * Parent double field = "dnum" + * Child string field = "pid" + * Child long field = "num" + * Child double field = "dnum" + */ + public void benchHasChildMatchAll(boolean cacheLookup) { + FilterBuilder lookupFilter = matchAllFilter(); + QueryBuilder mainQuery = matchAllQuery(); + + TermsLookupFilterBuilder stringFilter = termsLookupFilter("id") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("pid") + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder bloomFilter = termsLookupFilter("id") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("pid") + .bloomExpectedInsertions(NUM_PARENTS) + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder longFilter = termsLookupFilter("num") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("num") + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder doubleFilter = termsLookupFilter("dnum") + .index(CHILD_INDEX) + .type(CHILD_TYPE) + .path("dnum") + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + long tookString = 0; + long tookLong = 0; + long tookDouble = 0; + long tookBloom = 0; + long expected = NUM_PARENTS; + warmFieldData("id", "pid"); // for string fields + warmFieldData("num", "num"); // for long fields + warmFieldData("dnum", "dnum"); // for double fields + + log("==== HAS CHILD MATCH-ALL (cache: " + cacheLookup + ") ===="); + for (int i = 0; i < NUM_QUERIES; i++) { + tookString += runQuery("string", i, PARENT_INDEX, expected, filteredQuery(mainQuery, stringFilter)); + tookLong += runQuery("long", i, PARENT_INDEX, expected, filteredQuery(mainQuery, longFilter)); + tookDouble += runQuery("double", i, PARENT_INDEX, expected, filteredQuery(mainQuery, doubleFilter)); + tookBloom += runQuery("bloom", i, PARENT_INDEX, expected, filteredQuery(mainQuery, bloomFilter)); + } + + log("string: " + (tookString / NUM_QUERIES) + "ms avg"); + log("long : " + (tookLong / NUM_QUERIES) + "ms avg"); + log("double: " + (tookDouble / NUM_QUERIES) + "ms avg"); + log("bloom : " + (tookBloom / NUM_QUERIES) + "ms avg"); + log(""); + } + + /** + * Search for children that have a parent with the specified name. + * Expect NUM_CHILDREN_PER_PARENT since only one parent matching lookup. + *

+ * Parent string field = "id" + * Parent numeric field = "num" + * Child string field = "pid" + * Child numeric field = "num" + */ + public void benchHasParentSingleTerm(boolean cacheLookup) { + FilterBuilder lookupFilter; + QueryBuilder mainQuery = matchAllQuery(); + + TermsLookupFilterBuilder stringFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder bloomFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + .bloomExpectedInsertions(NUM_PARENTS) + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder longFilter = termsLookupFilter("num") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("num") + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder doubleFilter = termsLookupFilter("dnum") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("dnum") + .lookupCache(cacheLookup); + + long tookString = 0; + long tookLong = 0; + long tookDouble = 0; + long tookBloom = 0; + long expected = NUM_CHILDREN_PER_PARENT; + warmFieldData("id", "pid"); // for string fields + warmFieldData("num", "num"); // for long fields + warmFieldData("dnum", "dnum"); // for double fields + + log("==== HAS PARENT SINGLE TERM (cache: " + cacheLookup + ") ===="); + for (int i = 0; i < NUM_QUERIES; i++) { + lookupFilter = termFilter("name", "test" + (random.nextInt(NUM_PARENTS) + 1)); + + stringFilter.lookupFilter(lookupFilter); + longFilter.lookupFilter(lookupFilter); + doubleFilter.lookupFilter(lookupFilter); + bloomFilter.lookupFilter(lookupFilter); + + tookString += runQuery("string", i, CHILD_INDEX, expected, filteredQuery(mainQuery, stringFilter)); + tookLong += runQuery("long", i, CHILD_INDEX, expected, filteredQuery(mainQuery, longFilter)); + tookDouble += runQuery("double", i, CHILD_INDEX, expected, filteredQuery(mainQuery, doubleFilter)); + tookBloom += runQuery("bloom", i, CHILD_INDEX, expected, filteredQuery(mainQuery, bloomFilter)); + } + + log("string: " + (tookString / NUM_QUERIES) + "ms avg"); + log("long: " + (tookLong / NUM_QUERIES) + "ms avg"); + log("double: " + (tookDouble / NUM_QUERIES) + "ms avg"); + log("bloom: " + (tookBloom / NUM_QUERIES) + "ms avg"); + log(""); + } + + /** + * Search for children that have a parent. + * Expect all children to be returned. + *

+ * Parent string field = "id" + * Parent long field = "num" + * Parent double field = "dnum" + * Child string field = "pid" + * Child long field = "num" + * Child double field = "dnum" + */ + public void benchHasParentMatchAll(boolean cacheLookup) { + FilterBuilder lookupFilter = matchAllFilter(); + QueryBuilder mainQuery = matchAllQuery(); + + TermsLookupFilterBuilder stringFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder bloomFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + .bloomExpectedInsertions(NUM_PARENTS * NUM_CHILDREN_PER_PARENT) + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder longFilter = termsLookupFilter("num") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("num") + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder doubleFilter = termsLookupFilter("dnum") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("dnum") + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + long tookString = 0; + long tookLong = 0; + long tookDouble = 0; + long tookBloom = 0; + long expected = NUM_CHILDREN_PER_PARENT * NUM_PARENTS; + warmFieldData("id", "pid"); // for string fields + warmFieldData("num", "num"); // for numeric fields + warmFieldData("dnum", "dnum"); // for double fields + + log("==== HAS PARENT MATCH-ALL (cache: " + cacheLookup + ") ===="); + for (int i = 0; i < NUM_QUERIES; i++) { + tookString += runQuery("string", i, CHILD_INDEX, expected, filteredQuery(mainQuery, stringFilter)); + tookLong += runQuery("long", i, CHILD_INDEX, expected, filteredQuery(mainQuery, longFilter)); + tookDouble += runQuery("double", i, CHILD_INDEX, expected, filteredQuery(mainQuery, doubleFilter)); + tookBloom += runQuery("bloom", i, CHILD_INDEX, expected, filteredQuery(mainQuery, bloomFilter)); + } + + log("string: " + (tookString / NUM_QUERIES) + "ms avg"); + log("long : " + (tookLong / NUM_QUERIES) + "ms avg"); + log("double: " + (tookDouble / NUM_QUERIES) + "ms avg"); + log("bloom : " + (tookBloom / NUM_QUERIES) + "ms avg"); + log(""); + } + + /** + * Search for children that have a parent but limit each shard to 10k terms. + * Expect maxTermsPerShard * NUM_SHARDS * NUM_CHILDREN_PER_PARENT children returned. + * + * Parent string field = "id" + * Parent long field = "num" + * Parent double field = "dnum" + * Child string field = "pid" + * Child long field = "num" + * Child double field = "dnum" + */ + public void benchHasParentMatchAllWithMaxTerms(boolean cacheLookup) { + FilterBuilder lookupFilter = matchAllFilter(); + QueryBuilder mainQuery = matchAllQuery(); + long maxTermsPerShard = 30000; + + TermsLookupFilterBuilder stringFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + .maxTermsPerShard(maxTermsPerShard) + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder bloomFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + .maxTermsPerShard(maxTermsPerShard) + .bloomExpectedInsertions(NUM_PARENTS * NUM_CHILDREN_PER_PARENT) + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder longFilter = termsLookupFilter("num") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("num") + .maxTermsPerShard(maxTermsPerShard) + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + TermsLookupFilterBuilder doubleFilter = termsLookupFilter("dnum") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("dnum") + .maxTermsPerShard(maxTermsPerShard) + .lookupCache(cacheLookup) + .lookupFilter(lookupFilter); + + long tookString = 0; + long tookLong = 0; + long tookDouble = 0; + long tookBloom = 0; + long expected = NUM_CHILDREN_PER_PARENT * maxTermsPerShard * NUM_SHARDS; + warmFieldData("id", "pid"); // for string fields + warmFieldData("num", "num"); // for long fields + warmFieldData("dnum", "dnum"); // for double fields + + log("==== HAS PARENT MATCH-ALL (shard max terms: " + maxTermsPerShard + " cache: " + cacheLookup + ") ===="); + for (int i = 0; i < NUM_QUERIES; i++) { + tookString += runQuery("string", i, CHILD_INDEX, expected, filteredQuery(mainQuery, stringFilter)); + tookLong += runQuery("long", i, CHILD_INDEX, expected, filteredQuery(mainQuery, longFilter)); + tookDouble += runQuery("double", i, CHILD_INDEX, expected, filteredQuery(mainQuery, doubleFilter)); + tookBloom += runQuery("bloom", i, CHILD_INDEX, expected, filteredQuery(mainQuery, bloomFilter)); + } + + log("string: " + (tookString / NUM_QUERIES) + "ms avg"); + log("long : " + (tookLong / NUM_QUERIES) + "ms avg"); + log("double: " + (tookDouble / NUM_QUERIES) + "ms avg"); + log("bloom : " + (tookBloom / NUM_QUERIES) + "ms avg"); + log(""); + } + + /** + * Search for children that have a parent with any of the specified names. + * Expect NUM_CHILDREN_PER_PARENT * # of names. + *

+ * Parent string field = "id" + * Parent long field = "num" + * Parent double field = "dnum" + * Child string field = "pid" + * Child long field = "num" + * Child double field = "dnum" + */ + public void benchHasParentRandomTerms(boolean cacheLookup) { + FilterBuilder lookupFilter; + QueryBuilder mainQuery = matchAllQuery(); + Set names = new HashSet(NUM_PARENTS); + + TermsLookupFilterBuilder stringFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder bloomFilter = termsLookupFilter("pid") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("id") + + /* + Tune the bloom filter to give acceptable false positives for this test. + */ + .bloomHashFunctions(10) + .bloomFpp(0.01) + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder longFilter = termsLookupFilter("num") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("num") + .lookupCache(cacheLookup); + + TermsLookupFilterBuilder doubleFilter = termsLookupFilter("dnum") + .index(PARENT_INDEX) + .type(PARENT_TYPE) + .path("dnum") + .lookupCache(cacheLookup); + + long tookString = 0; + long tookLong = 0; + long tookDouble = 0; + long tookBloom = 0; + int expected = 0; + warmFieldData("id", "pid"); // for string fields + warmFieldData("num", "num"); // for long fields + warmFieldData("dnum", "dnum"); // for double fields + warmFieldData("name", null); // for field data terms filter + + log("==== HAS PARENT RANDOM TERMS (cache: " + cacheLookup + ") ===="); + for (int i = 0; i < NUM_QUERIES; i++) { + + // add a random number of terms to the set on each iteration + int randNum = random.nextInt(NUM_PARENTS / NUM_QUERIES) + 1; + for (int j = 0; j < randNum; j++) { + names.add("test" + (random.nextInt(NUM_PARENTS) + 1)); + } + + lookupFilter = termsFilter("name", names).execution("fielddata"); + expected = NUM_CHILDREN_PER_PARENT * names.size(); + stringFilter.lookupFilter(lookupFilter); + longFilter.lookupFilter(lookupFilter); + doubleFilter.lookupFilter(lookupFilter); + bloomFilter.lookupFilter(lookupFilter).bloomExpectedInsertions(expected); // part of bloom filter tuning + + tookString += runQuery("string", i, CHILD_INDEX, expected, filteredQuery(mainQuery, stringFilter)); + tookLong += runQuery("long", i, CHILD_INDEX, expected, filteredQuery(mainQuery, longFilter)); + tookDouble += runQuery("double", i, CHILD_INDEX, expected, filteredQuery(mainQuery, doubleFilter)); + tookBloom += runQuery("bloom", i, CHILD_INDEX, expected, filteredQuery(mainQuery, bloomFilter)); + } + + log("string: " + (tookString / NUM_QUERIES) + "ms avg"); + log("long : " + (tookLong / NUM_QUERIES) + "ms avg"); + log("double: " + (tookDouble / NUM_QUERIES) + "ms avg"); + log("bloom : " + (tookBloom / NUM_QUERIES) + "ms avg"); + log(""); + } +} diff --git a/src/test/java/org/elasticsearch/count/query/SimpleQueryTests.java b/src/test/java/org/elasticsearch/count/query/SimpleQueryTests.java index dafaac22be02e..da3b5d8eed657 100644 --- a/src/test/java/org/elasticsearch/count/query/SimpleQueryTests.java +++ b/src/test/java/org/elasticsearch/count/query/SimpleQueryTests.java @@ -631,47 +631,46 @@ public void testTermsLookupFilter() throws Exception { client().prepareIndex("test", "type", "4").setSource("term", "4")); CountResponse countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("1").path("terms"))).get(); assertHitCount(countResponse, 2l); // same as above, just on the _id... countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("_id").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("_id").index("lookup").type("type").id("1").path("terms"))).get(); assertHitCount(countResponse, 2l); // another search with same parameters... countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("1").path("terms"))).get(); assertHitCount(countResponse, 2l); countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("2").lookupPath("terms"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("2").path("terms"))).get(); assertHitCount(countResponse, 1l); countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("3").lookupPath("terms")) - ).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("3").path("terms"))).get(); assertNoFailures(countResponse); assertHitCount(countResponse, 2l); countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("4").lookupPath("terms"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("4").path("terms"))).get(); assertHitCount(countResponse, 0l); countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup2").lookupType("type").lookupId("1").lookupPath("arr.term"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup2").type("type").id("1").path("arr.term"))).get(); assertHitCount(countResponse, 2l); countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup2").lookupType("type").lookupId("2").lookupPath("arr.term"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup2").type("type").id("2").path("arr.term"))).get(); assertHitCount(countResponse, 1l); countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup2").lookupType("type").lookupId("3").lookupPath("arr.term"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup2").type("type").id("3").path("arr.term"))).get(); assertHitCount(countResponse, 2l); countResponse = client().prepareCount("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("not_exists").lookupIndex("lookup2").lookupType("type").lookupId("3").lookupPath("arr.term"))).get(); + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("not_exists").index("lookup2").type("type").id("3").path("arr.term"))).get(); assertHitCount(countResponse, 0l); } diff --git a/src/test/java/org/elasticsearch/search/query/SimpleQueryTests.java b/src/test/java/org/elasticsearch/search/query/SimpleQueryTests.java index a370a48b62f53..d28dfc8cc2955 100644 --- a/src/test/java/org/elasticsearch/search/query/SimpleQueryTests.java +++ b/src/test/java/org/elasticsearch/search/query/SimpleQueryTests.java @@ -1003,6 +1003,10 @@ public void testTermsLookupFilter() throws Exception { jsonBuilder().startObject().startObject("type").startObject("properties") .startObject("arr").startObject("properties").startObject("term").field("type", "string") .endObject().endObject().endObject().endObject().endObject().endObject())); + assertAcked(prepareCreate("lookup3").addMapping("type", + jsonBuilder().startObject().startObject("type").startObject("properties") + .startObject("terms").field("type", "string").endObject() + .endObject().endObject().endObject())); assertAcked(prepareCreate("test").addMapping("type", "term", "type=string")); ensureGreen(); @@ -1032,65 +1036,92 @@ public void testTermsLookupFilter() throws Exception { client().prepareIndex("test", "type", "1").setSource("term", "1"), client().prepareIndex("test", "type", "2").setSource("term", "2"), client().prepareIndex("test", "type", "3").setSource("term", "3"), - client().prepareIndex("test", "type", "4").setSource("term", "4") ); + client().prepareIndex("test", "type", "4").setSource("term", "4"), + client().prepareIndex("test", "type", "5").setSource("term", "5"), + client().prepareIndex("lookup3", "type", "5").setSource("terms", new String[]{"5"}) ); + // terms lookup by filter SearchResponse searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").path("terms") + .lookupFilter(termsFilter("terms", "2", "4")).lookupCache(false)) + ).get(); + assertHitCount(searchResponse, 2l); + assertSearchHits(searchResponse, "2", "4"); + + // terms lookup by query + searchResponse = client().prepareSearch("test") + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").path("terms") + .lookupFilter(matchAllFilter()).lookupCache(false)) + ).get(); + assertHitCount(searchResponse, 4l); + assertSearchHits(searchResponse, "1", "2", "3", "4"); + + // terms lookup multi-index search + searchResponse = client().prepareSearch("test") + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").indices("lookup", "lookup3") + .type("type").path("terms").lookupFilter(termsFilter("terms", "1", "3", "5")).lookupCache(false)) + ).get(); + assertHitCount(searchResponse, 3l); + assertSearchHits(searchResponse, "1", "3", "5"); + + // terms lookup from terms stored in an indexed doc + searchResponse = client().prepareSearch("test") + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("1").path("terms")) ).get(); assertHitCount(searchResponse, 2l); assertSearchHits(searchResponse, "1", "3"); // same as above, just on the _id... searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("_id").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("_id").index("lookup").type("type").id("1").path("terms")) ).get(); assertHitCount(searchResponse, 2l); assertSearchHits(searchResponse, "1", "3"); // another search with same parameters... searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("1").lookupPath("terms")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("1").path("terms")) ).get(); assertHitCount(searchResponse, 2l); assertSearchHits(searchResponse, "1", "3"); searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("2").lookupPath("terms")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("2").path("terms")) ).get(); assertHitCount(searchResponse, 1l); assertFirstHit(searchResponse, hasId("2")); searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("3").lookupPath("terms")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("3").path("terms")) ).get(); assertHitCount(searchResponse, 2l); assertSearchHits(searchResponse, "2", "4"); searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup").lookupType("type").lookupId("4").lookupPath("terms")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup").type("type").id("4").path("terms")) ).get(); assertHitCount(searchResponse, 0l); searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup2").lookupType("type").lookupId("1").lookupPath("arr.term")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup2").type("type").id("1").path("arr.term")) ).get(); assertHitCount(searchResponse, 2l); assertSearchHits(searchResponse, "1", "3"); searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup2").lookupType("type").lookupId("2").lookupPath("arr.term")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup2").type("type").id("2").path("arr.term")) ).get(); assertHitCount(searchResponse, 1l); assertFirstHit(searchResponse, hasId("2")); searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").lookupIndex("lookup2").lookupType("type").lookupId("3").lookupPath("arr.term")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("term").index("lookup2").type("type").id("3").path("arr.term")) ).get(); assertHitCount(searchResponse, 2l); assertSearchHits(searchResponse, "2", "4"); searchResponse = client().prepareSearch("test") - .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("not_exists").lookupIndex("lookup2").lookupType("type").lookupId("3").lookupPath("arr.term")) + .setQuery(filteredQuery(matchAllQuery(), termsLookupFilter("not_exists").index("lookup2").type("type").id("3").path("arr.term")) ).get(); assertHitCount(searchResponse, 0l); }