Remove the terms API (replaced by facets in search), closes #242.

This commit is contained in:
kimchy 2010-06-30 19:49:41 +03:00
parent 53b66fb26a
commit 66096e97f2
35 changed files with 2 additions and 2773 deletions

View File

@ -53,7 +53,6 @@ import org.elasticsearch.action.mlt.TransportMoreLikeThisAction;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.search.TransportSearchScrollAction;
import org.elasticsearch.action.search.type.*;
import org.elasticsearch.action.terms.TransportTermsAction;
import org.elasticsearch.common.inject.AbstractModule;
/**
@ -95,7 +94,6 @@ public class TransportActionModule extends AbstractModule {
bind(TransportGetAction.class).asEagerSingleton();
bind(TransportDeleteAction.class).asEagerSingleton();
bind(TransportCountAction.class).asEagerSingleton();
bind(TransportTermsAction.class).asEagerSingleton();
bind(TransportShardDeleteByQueryAction.class).asEagerSingleton();
bind(TransportIndexDeleteByQueryAction.class).asEagerSingleton();

View File

@ -1,123 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.terms;
import org.elasticsearch.common.collect.Iterators;
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.trove.ExtTObjectIntHasMap;
import java.io.IOException;
import java.util.Iterator;
import static org.elasticsearch.action.terms.TermFreq.*;
/**
* All the {@link TermFreq}s that occur in a specific field.
*
* @author kimchy (shay.banon)
*/
public class FieldTermsFreq implements Streamable, Iterable<TermFreq> {
private String fieldName;
private TermFreq[] termsFreqs;
private transient ExtTObjectIntHasMap<Object> termsFreqMap;
private FieldTermsFreq() {
}
FieldTermsFreq(String fieldName, TermFreq[] termsFreqs) {
this.fieldName = fieldName;
this.termsFreqs = termsFreqs;
}
/**
* The field name.
*/
public String fieldName() {
return this.fieldName;
}
/**
* The field name.
*/
public String getFieldName() {
return fieldName;
}
/**
* The term frequencies of the field.
*/
public TermFreq[] termsFreqs() {
return this.termsFreqs;
}
/**
* The term frequencies of the field.
*/
public TermFreq[] getTermsFreqs() {
return termsFreqs;
}
/**
* Returns the document frequency of a term, <tt>-1</tt> if the term does not exists.
*/
public int docFreq(Object term) {
// we use "toString" on the term so we get hits when we the termValue is Long, and we lookup with int
if (termsFreqMap == null) {
ExtTObjectIntHasMap<Object> termsFreqMap = new ExtTObjectIntHasMap<Object>().defaultReturnValue(-1);
for (TermFreq termFreq : termsFreqs) {
termsFreqMap.put(termFreq.term().toString(), termFreq.docFreq());
}
this.termsFreqMap = termsFreqMap;
}
return termsFreqMap.get(term.toString());
}
@Override public Iterator<TermFreq> iterator() {
return Iterators.forArray(termsFreqs);
}
public static FieldTermsFreq readFieldTermsFreq(StreamInput in) throws IOException {
FieldTermsFreq fieldTermsFreq = new FieldTermsFreq();
fieldTermsFreq.readFrom(in);
return fieldTermsFreq;
}
@Override public void readFrom(StreamInput in) throws IOException {
fieldName = in.readUTF();
termsFreqs = new TermFreq[in.readVInt()];
for (int i = 0; i < termsFreqs.length; i++) {
termsFreqs[i] = readTermFreq(in);
}
}
@Override public void writeTo(StreamOutput out) throws IOException {
out.writeUTF(fieldName);
out.writeVInt(termsFreqs.length);
for (TermFreq termFreq : termsFreqs) {
termFreq.writeTo(out);
}
}
}

View File

@ -1,171 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.terms;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
/**
* @author kimchy (shay.banon)
*/
class ShardTermsRequest extends BroadcastShardOperationRequest {
private String[] fields;
private String from;
private String to;
private boolean fromInclusive = true;
private boolean toInclusive = true;
private String prefix;
private String regexp;
private int size = 10;
private TermsRequest.SortType sortType;
private boolean exact = false;
ShardTermsRequest() {
}
public ShardTermsRequest(String index, int shardId, TermsRequest request) {
super(index, shardId);
this.fields = request.fields();
this.from = request.from();
this.to = request.to();
this.fromInclusive = request.fromInclusive();
this.toInclusive = request.toInclusive();
this.prefix = request.prefix();
this.regexp = request.regexp();
this.size = request.size();
this.sortType = request.sortType();
this.exact = request.exact();
}
public String[] fields() {
return fields;
}
public String from() {
return from;
}
public String to() {
return to;
}
public boolean fromInclusive() {
return fromInclusive;
}
public boolean toInclusive() {
return toInclusive;
}
public String prefix() {
return prefix;
}
public String regexp() {
return regexp;
}
public int size() {
return size;
}
public TermsRequest.SortType sortType() {
return sortType;
}
public boolean exact() {
return this.exact;
}
@Override public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
fields = new String[in.readVInt()];
for (int i = 0; i < fields.length; i++) {
fields[i] = in.readUTF();
}
if (in.readBoolean()) {
from = in.readUTF();
}
if (in.readBoolean()) {
to = in.readUTF();
}
fromInclusive = in.readBoolean();
toInclusive = in.readBoolean();
if (in.readBoolean()) {
prefix = in.readUTF();
}
if (in.readBoolean()) {
regexp = in.readUTF();
}
size = in.readVInt();
sortType = TermsRequest.SortType.fromValue(in.readByte());
exact = in.readBoolean();
}
@Override public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(fields.length);
for (String field : fields) {
out.writeUTF(field);
}
if (from == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(from);
}
if (to == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(to);
}
out.writeBoolean(fromInclusive);
out.writeBoolean(toInclusive);
if (prefix == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(prefix);
}
if (regexp == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(regexp);
}
out.writeVInt(size);
out.writeByte(sortType.value());
out.writeBoolean(exact);
}
}

View File

@ -1,111 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.terms;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.trove.TObjectIntHashMap;
import org.elasticsearch.common.trove.TObjectIntIterator;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
/**
* @author kimchy (shay.banon)
*/
class ShardTermsResponse extends BroadcastShardOperationResponse {
private Map<String, TObjectIntHashMap<Object>> fieldsTermsFreqs = new HashMap<String, TObjectIntHashMap<Object>>();
private int numDocs;
private int maxDoc;
private int numDeletedDocs;
ShardTermsResponse() {
}
ShardTermsResponse(String index, int shardId, int numDocs, int maxDoc, int numDeletedDocs) {
super(index, shardId);
this.numDocs = numDocs;
this.maxDoc = maxDoc;
this.numDeletedDocs = numDeletedDocs;
}
int numDocs() {
return this.numDocs;
}
int maxDoc() {
return this.maxDoc;
}
int numDeletedDocs() {
return this.numDeletedDocs;
}
void put(String fieldName, TObjectIntHashMap<Object> termsFreqs) {
fieldsTermsFreqs.put(fieldName, termsFreqs);
}
Map<String, TObjectIntHashMap<Object>> fieldsTermsFreqs() {
return fieldsTermsFreqs;
}
@Override public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
numDocs = in.readVInt();
maxDoc = in.readVInt();
numDeletedDocs = in.readVInt();
int size = in.readVInt();
for (int i = 0; i < size; i++) {
String fieldName = in.readUTF();
TObjectIntHashMap<Object> termsFreq = new TObjectIntHashMap<Object>();
int size1 = in.readVInt();
for (int j = 0; j < size1; j++) {
termsFreq.put(Lucene.readFieldValue(in), in.readVInt());
}
fieldsTermsFreqs.put(fieldName, termsFreq);
}
}
@Override public void writeTo(final StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(numDocs);
out.writeVInt(maxDoc);
out.writeVInt(numDeletedDocs);
out.writeVInt(fieldsTermsFreqs.size());
for (Map.Entry<String, TObjectIntHashMap<Object>> entry : fieldsTermsFreqs.entrySet()) {
out.writeUTF(entry.getKey());
out.writeVInt(entry.getValue().size());
for (TObjectIntIterator<Object> it = entry.getValue().iterator(); it.hasNext();) {
it.advance();
Lucene.writeFieldValue(out, it.key());
out.writeVInt(it.value());
}
}
}
}

View File

@ -1,143 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.terms;
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.lucene.Lucene;
import java.io.IOException;
import java.util.Comparator;
/**
* A tuple of term and its document frequency (in how many documents this term exists).
*
* @author kimchy (shay.banon)
*/
public class TermFreq implements Streamable {
/**
* A frequency based comparator with higher frequencies first.
*/
private static final Comparator<TermFreq> freqComparator = new Comparator<TermFreq>() {
@Override public int compare(TermFreq o1, TermFreq o2) {
int i = o2.docFreq() - o1.docFreq();
if (i == 0) {
i = ((Comparable) o1.term()).compareTo(o2.term());
}
return i;
}
};
/**
* Lexical based comparator.
*/
private static final Comparator<TermFreq> termComparator = new Comparator<TermFreq>() {
@Override public int compare(TermFreq o1, TermFreq o2) {
int i = ((Comparable) o1.term()).compareTo(o2.term());
if (i == 0) {
i = o1.docFreq() - o2.docFreq();
}
return i;
}
};
/**
* A frequency based comparator with higher frequencies first.
*/
public static Comparator<TermFreq> freqComparator() {
return freqComparator;
}
/**
* Lexical based comparator.
*/
public static Comparator<TermFreq> termComparator() {
return termComparator;
}
private Object term;
private int docFreq;
private TermFreq() {
}
/**
* Constructs a new term freq.
*
* @param term The term
* @param docFreq The document frequency
*/
TermFreq(Object term, int docFreq) {
this.term = term;
this.docFreq = docFreq;
}
/**
* The term.
*/
public Object term() {
return term;
}
/**
* The term.
*/
public Object getTerm() {
return term;
}
public String termAsString() {
return term.toString();
}
/**
* The document frequency of the term (in how many documents this term exists).
*/
public int docFreq() {
return docFreq;
}
/**
* The document frequency of the term (in how many documents this term exists).
*/
public int getDocFreq() {
return docFreq;
}
public static TermFreq readTermFreq(StreamInput in) throws IOException {
TermFreq termFreq = new TermFreq();
termFreq.readFrom(in);
return termFreq;
}
@Override public void readFrom(StreamInput in) throws IOException {
term = Lucene.readFieldValue(in);
docFreq = in.readVInt();
}
@Override public void writeTo(StreamOutput out) throws IOException {
Lucene.writeFieldValue(out, term);
out.writeVInt(docFreq);
}
}

View File

@ -1,467 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.terms;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.broadcast.BroadcastOperationRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.mapper.AllFieldMapper;
import java.io.IOException;
/**
* Terms request represent a request to get terms in one or more indices of specific fields and their
* document frequencies (in how many document each term exists).
*
* <p>By default, the "_all" field will be used to extract terms and frequencies.
*
* <p>This is very handy to implement things like tag clouds and auto complete (using {@link #prefix(String)} or
* {@link #regexp(String)}).
*
* @author kimchy (shay.banon)
*/
public class TermsRequest extends BroadcastOperationRequest {
/**
* The type of sorting for terms.
*/
public static enum SortType {
/**
* Sort based on the term (lex).
*/
TERM((byte) 0),
/**
* Sort based on the term document frequency.
*/
FREQ((byte) 1);
private byte value;
SortType(byte value) {
this.value = value;
}
/**
* The unique byte value of the sort type.
*/
public byte value() {
return value;
}
/**
* Parses the sort type from its {@link #value()}.
*/
public static SortType fromValue(byte value) {
switch (value) {
case 0:
return TERM;
case 1:
return FREQ;
default:
throw new ElasticSearchIllegalArgumentException("No value for [" + value + "]");
}
}
/**
* Parses the sort type from a string. Can either be "term" or "freq". If <tt>null</tt>
* is passed, will return the defaultSort provided.
*
* @param value The string value to parse. Can be either "term" or "freq"
* @param defaultSort The sort type to return in case value is <tt>null</tt>
* @return The sort type parsed
*/
public static SortType fromString(String value, SortType defaultSort) {
if (value == null) {
return defaultSort;
}
if (value.equals("term")) {
return TERM;
} else if (value.equals("freq")) {
return FREQ;
} else {
throw new ElasticSearchIllegalArgumentException("Illegal sort type [" + value + "], must be one of [term,freq]");
}
}
}
private static final String[] DEFAULT_FIELDS = new String[]{AllFieldMapper.NAME};
private String[] fields = DEFAULT_FIELDS;
private String from;
private boolean fromInclusive = true;
private String to;
private boolean toInclusive = true;
private String prefix;
private String regexp;
private int minFreq = 1;
private int maxFreq = Integer.MAX_VALUE;
private int size = 10;
private boolean convert = true;
private SortType sortType = SortType.TERM;
private boolean exact = false;
TermsRequest() {
}
/**
* Constructs a new terms requests with the provided indices. Don't pass anything for it to run
* over all the indices.
*/
public TermsRequest(String... indices) {
super(indices, null);
}
@Override public ActionRequestValidationException validate() {
ActionRequestValidationException validationException = super.validate();
if (fields == null || fields.length == 0) {
fields = DEFAULT_FIELDS;
}
return validationException;
}
/**
* The fields within each document which terms will be iterated over and returned with the
* document frequencies.
*/
public String[] fields() {
return this.fields;
}
/**
* The fields within each document which terms will be iterated over and returned with the
* document frequencies. By default will use the "_all" field.
*/
public TermsRequest fields(String... fields) {
this.fields = fields;
return this;
}
/**
* The lower bound term from which the iteration will start. Defaults to start from the
* first.
*/
public String from() {
return from;
}
/**
* The lower bound term from which the iteration will start. Defaults to start from the
* first.
*/
public TermsRequest from(Object from) {
if (from == null) {
this.from = null;
} else {
this.from = from.toString();
}
return this;
}
/**
* Greater than (like setting from with fromIInclusive set to <tt>false</tt>).
*/
public TermsRequest gt(Object from) {
from(from);
fromInclusive(false);
return this;
}
/**
* Greater/equal than (like setting from with fromInclusive set to <tt>true</tt>).
*/
public TermsRequest gte(Object from) {
from(from);
fromInclusive(true);
return this;
}
/**
* Lower then (like setting to with toInclusive set to <tt>false</tt>)
*/
public TermsRequest lt(Object to) {
to(to);
toInclusive(false);
return this;
}
/**
* Lower/equal then (like setting to with toInclusive set to <tt>false</tt>)
*/
public TermsRequest lte(Object to) {
to(to);
toInclusive(true);
return this;
}
/**
* Should the first from (if set using {@link #from(Object)} be inclusive or not. Defaults
* to <tt>false</tt> (not inclusive / exclusive).
*/
public boolean fromInclusive() {
return fromInclusive;
}
/**
* Should the first from (if set using {@link #from(Object)} be inclusive or not. Defaults
* to <tt>false</tt> (not inclusive / exclusive).
*/
public TermsRequest fromInclusive(boolean fromInclusive) {
this.fromInclusive = fromInclusive;
return this;
}
/**
* The upper bound term to which the iteration will end. Defaults to unbound (<tt>null</tt>).
*/
public String to() {
return to;
}
/**
* The upper bound term to which the iteration will end. Defaults to unbound (<tt>null</tt>).
*/
public TermsRequest to(Object to) {
if (to == null) {
this.to = null;
} else {
this.to = to.toString();
}
return this;
}
/**
* Should the last to (if set using {@link #to(Object)} be inclusive or not. Defaults to
* <tt>true</tt>.
*/
public boolean toInclusive() {
return toInclusive;
}
/**
* Should the last to (if set using {@link #to(Object)} be inclusive or not. Defaults to
* <tt>true</tt>.
*/
public TermsRequest toInclusive(boolean toInclusive) {
this.toInclusive = toInclusive;
return this;
}
/**
* An optional prefix from which the terms iteration will start (in lex order).
*/
public String prefix() {
return prefix;
}
/**
* An optional prefix from which the terms iteration will start (in lex order).
*/
public TermsRequest prefix(String prefix) {
this.prefix = prefix;
return this;
}
/**
* An optional regular expression to filter out terms (only the ones that match the regexp
* will return).
*/
public String regexp() {
return regexp;
}
/**
* An optional regular expression to filter out terms (only the ones that match the regexp
* will return).
*/
public TermsRequest regexp(String regexp) {
this.regexp = regexp;
return this;
}
/**
* An optional minimum document frequency to filter out terms.
*/
public int minFreq() {
return minFreq;
}
/**
* An optional minimum document frequency to filter out terms.
*/
public TermsRequest minFreq(int minFreq) {
this.minFreq = minFreq;
return this;
}
/**
* An optional maximum document frequency to filter out terms.
*/
public int maxFreq() {
return maxFreq;
}
/**
* An optional maximum document frequency to filter out terms.
*/
public TermsRequest maxFreq(int maxFreq) {
this.maxFreq = maxFreq;
return this;
}
/**
* The number of term / doc freq pairs to return per field. Defaults to <tt>10</tt>.
*/
public int size() {
return size;
}
/**
* The number of term / doc freq pairs to return per field. Defaults to <tt>10</tt>.
*/
public TermsRequest size(int size) {
this.size = size;
return this;
}
/**
* The type of sorting for term / doc freq. Can either sort on term (lex) or doc frequency. Defaults to
* {@link TermsRequest.SortType#TERM}.
*/
public SortType sortType() {
return sortType;
}
/**
* The type of sorting for term / doc freq. Can either sort on term (lex) or doc frequency. Defaults to
* {@link TermsRequest.SortType#TERM}.
*/
public TermsRequest sortType(SortType sortType) {
this.sortType = sortType;
return this;
}
/**
* Sets the string representation of the sort type.
*/
public TermsRequest sortType(String sortType) {
return sortType(SortType.fromString(sortType, this.sortType));
}
/**
* Should the doc frequencies be exact frequencies. Exact frequencies takes into account deletes that
* have not been merged and cleaned (optimized). Note, when this is set to <tt>true</tt> this operation
* might be an expensive operation. Defaults to <tt>false</tt>.
*/
public boolean exact() {
return exact;
}
/**
* Should the doc frequencies be exact frequencies. Exact frequencies takes into account deletes that
* have not been merged and cleaned (optimized). Note, when this is set to <tt>true</tt> this operation
* might be an expensive operation. Defaults to <tt>false</tt>.
*/
public TermsRequest exact(boolean exact) {
this.exact = exact;
return this;
}
@Override public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVInt(fields.length);
for (String field : fields) {
out.writeUTF(field);
}
if (from == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(from);
}
if (to == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(to);
}
out.writeBoolean(fromInclusive);
out.writeBoolean(toInclusive);
if (prefix == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(prefix);
}
if (regexp == null) {
out.writeBoolean(false);
} else {
out.writeBoolean(true);
out.writeUTF(regexp);
}
out.writeVInt(size);
out.writeBoolean(convert);
out.writeByte(sortType.value());
out.writeVInt(minFreq);
out.writeVInt(maxFreq);
out.writeBoolean(exact);
}
@Override public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
fields = new String[in.readVInt()];
for (int i = 0; i < fields.length; i++) {
fields[i] = in.readUTF();
}
if (in.readBoolean()) {
from = in.readUTF();
}
if (in.readBoolean()) {
to = in.readUTF();
}
fromInclusive = in.readBoolean();
toInclusive = in.readBoolean();
if (in.readBoolean()) {
prefix = in.readUTF();
}
if (in.readBoolean()) {
regexp = in.readUTF();
}
size = in.readVInt();
convert = in.readBoolean();
sortType = TermsRequest.SortType.fromValue(in.readByte());
minFreq = in.readVInt();
maxFreq = in.readVInt();
exact = in.readBoolean();
}
}

View File

@ -1,173 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.terms;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse;
import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.elasticsearch.action.terms.FieldTermsFreq.*;
/**
* The response of terms request. Includes a list of {@link FieldTermsFreq} which include
* the field and all its term / doc freq pair.
*
* @author kimchy (shay.banon)
*/
public class TermsResponse extends BroadcastOperationResponse implements Iterable<FieldTermsFreq> {
private long numDocs;
private long maxDoc;
private long numDeletedDocs;
private FieldTermsFreq[] fieldsTermsFreq;
private transient Map<String, FieldTermsFreq> fieldsTermsFreqMap;
TermsResponse() {
}
TermsResponse(int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures, FieldTermsFreq[] fieldsTermsFreq,
long numDocs, long maxDoc, long numDeletedDocs) {
super(totalShards, successfulShards, failedShards, shardFailures);
this.fieldsTermsFreq = fieldsTermsFreq;
this.numDocs = numDocs;
this.maxDoc = maxDoc;
this.numDeletedDocs = numDeletedDocs;
}
/**
* The total number of documents.
*/
public long numDocs() {
return this.numDocs;
}
/**
* The total number of documents.
*/
public long getNumDocs() {
return numDocs;
}
/**
* The total maximum number of documents (including deletions).
*/
public long maxDoc() {
return this.maxDoc;
}
/**
* The total maximum number of documents (including deletions).
*/
public long getMaxDoc() {
return maxDoc;
}
/**
* The number of deleted docs.
*/
public long deletedDocs() {
return this.numDeletedDocs;
}
/**
* The number of deleted docs.
*/
public long getNumDeletedDocs() {
return numDeletedDocs;
}
/**
* Iterates over the {@link FieldTermsFreq}.
*/
@Override public Iterator<FieldTermsFreq> iterator() {
return Iterators.forArray(fieldsTermsFreq);
}
/**
* The {@link FieldTermsFreq} for the specified field name, <tt>null</tt> if
* there is none.
*
* @param fieldName The field name to return the field terms freq for
* @return The field terms freq
*/
public FieldTermsFreq field(String fieldName) {
return fieldsAsMap().get(fieldName);
}
/**
* All the {@link FieldTermsFreq}.
*/
public FieldTermsFreq[] fields() {
return this.fieldsTermsFreq;
}
public Map<String, FieldTermsFreq> getFields() {
return fieldsAsMap();
}
/**
* The pair of field name to {@link FieldTermsFreq} as map for simpler usage.
*/
public Map<String, FieldTermsFreq> fieldsAsMap() {
if (fieldsTermsFreqMap != null) {
return fieldsTermsFreqMap;
}
Map<String, FieldTermsFreq> fieldsTermsFreqMap = new HashMap<String, FieldTermsFreq>();
for (FieldTermsFreq fieldTermsFreq : fieldsTermsFreq) {
fieldsTermsFreqMap.put(fieldTermsFreq.fieldName(), fieldTermsFreq);
}
this.fieldsTermsFreqMap = fieldsTermsFreqMap;
return fieldsTermsFreqMap;
}
@Override public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
numDocs = in.readVLong();
maxDoc = in.readVLong();
numDeletedDocs = in.readVLong();
fieldsTermsFreq = new FieldTermsFreq[in.readVInt()];
for (int i = 0; i < fieldsTermsFreq.length; i++) {
fieldsTermsFreq[i] = readFieldTermsFreq(in);
}
}
@Override public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeVLong(numDocs);
out.writeVLong(maxDoc);
out.writeVLong(numDeletedDocs);
out.writeVInt(fieldsTermsFreq.length);
for (FieldTermsFreq fieldTermsFreq : fieldsTermsFreq) {
fieldTermsFreq.writeTo(out);
}
}
}

View File

@ -1,430 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.terms;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.TermDocs;
import org.apache.lucene.index.TermEnum;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.StringHelper;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.TransportActions;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.TransportBroadcastOperationAction;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.collect.BoundedTreeSet;
import org.elasticsearch.common.collect.Maps;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.trove.TObjectIntHashMap;
import org.elasticsearch.common.trove.TObjectIntIterator;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.FieldMapper;
import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.atomic.AtomicReferenceArray;
import java.util.regex.Pattern;
import static org.elasticsearch.common.collect.Lists.*;
/**
* @author kimchy (shay.banon)
*/
public class TransportTermsAction extends TransportBroadcastOperationAction<TermsRequest, TermsResponse, ShardTermsRequest, ShardTermsResponse> {
@Inject public TransportTermsAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, IndicesService indicesService) {
super(settings, threadPool, clusterService, transportService, indicesService);
}
@Override protected TermsResponse newResponse(final TermsRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) {
int successfulShards = 0;
int failedShards = 0;
long numDocs = 0;
long maxDoc = 0;
long numDeletedDocs = 0;
List<ShardOperationFailedException> shardFailures = null;
Map<String, TObjectIntHashMap<Object>> aggregator = Maps.newHashMap();
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
failedShards++;
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = newArrayList();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
ShardTermsResponse shardTermsResponse = (ShardTermsResponse) shardResponse;
IndexService indexService = indicesService.indexServiceSafe(shardTermsResponse.index());
for (Map.Entry<String, TObjectIntHashMap<Object>> entry : shardTermsResponse.fieldsTermsFreqs().entrySet()) {
String fieldName = entry.getKey();
FieldMapper fieldMapper = indexService.mapperService().smartNameFieldMapper(fieldName);
TObjectIntHashMap<Object> termsFreqs = aggregator.get(fieldName);
if (termsFreqs == null) {
termsFreqs = new TObjectIntHashMap<Object>();
aggregator.put(fieldName, termsFreqs);
}
for (TObjectIntIterator<Object> it = entry.getValue().iterator(); it.hasNext();) {
it.advance();
Object termValue = it.key();
int freq = it.value();
if (fieldMapper != null) {
termValue = fieldMapper.valueForSearch(termValue);
}
termsFreqs.adjustOrPutValue(termValue, freq, freq);
}
}
numDocs += shardTermsResponse.numDocs();
maxDoc += shardTermsResponse.maxDoc();
numDeletedDocs += shardTermsResponse.numDeletedDocs();
successfulShards++;
}
}
Map<String, NavigableSet<TermFreq>> fieldTermsFreqs = new HashMap<String, NavigableSet<TermFreq>>();
if (aggregator != null) {
for (Map.Entry<String, TObjectIntHashMap<Object>> entry : aggregator.entrySet()) {
String fieldName = entry.getKey();
NavigableSet<TermFreq> sortedFreqs = fieldTermsFreqs.get(fieldName);
if (sortedFreqs == null) {
Comparator<TermFreq> comparator = request.sortType() == TermsRequest.SortType.FREQ ? TermFreq.freqComparator() : TermFreq.termComparator();
sortedFreqs = new BoundedTreeSet<TermFreq>(comparator, request.size());
fieldTermsFreqs.put(fieldName, sortedFreqs);
}
for (TObjectIntIterator<Object> it = entry.getValue().iterator(); it.hasNext();) {
it.advance();
if (it.value() >= request.minFreq() && it.value() <= request.maxFreq()) {
sortedFreqs.add(new TermFreq(it.key(), it.value()));
}
}
}
}
FieldTermsFreq[] resultFreqs = new FieldTermsFreq[fieldTermsFreqs.size()];
int index = 0;
for (Map.Entry<String, NavigableSet<TermFreq>> entry : fieldTermsFreqs.entrySet()) {
TermFreq[] freqs = entry.getValue().toArray(new TermFreq[entry.getValue().size()]);
resultFreqs[index++] = new FieldTermsFreq(entry.getKey(), freqs);
}
return new TermsResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures, resultFreqs, numDocs, maxDoc, numDeletedDocs);
}
@Override protected ShardTermsResponse shardOperation(ShardTermsRequest request) throws ElasticSearchException {
IndexService indexService = indicesService.indexServiceSafe(request.index());
IndexShard shard = indexService.shard(request.shardId());
Engine.Searcher searcher = shard.searcher();
ShardTermsResponse response = new ShardTermsResponse(request.index(), request.shardId(),
searcher.reader().numDocs(), searcher.reader().maxDoc(), searcher.reader().numDeletedDocs());
TermDocs termDocs = null;
try {
Pattern regexpPattern = null;
if (request.regexp() != null) {
regexpPattern = Pattern.compile(request.regexp(), Pattern.DOTALL | Pattern.CASE_INSENSITIVE);
}
for (String fieldName : request.fields()) {
FieldMapper fieldMapper = indexService.mapperService().smartNameFieldMapper(fieldName);
String indexFieldName = fieldName;
if (fieldMapper != null) {
indexFieldName = fieldMapper.names().indexName();
}
indexFieldName = StringHelper.intern(indexFieldName);
// if we are sorting by term, and the field mapper sorting type is STRING, then do plain term extraction (which is faster)
try {
ExecuteTermResult executeTermResult;
if (request.sortType() == TermsRequest.SortType.TERM && fieldMapper != null && (fieldMapper.sortType() == SortField.STRING || fieldMapper.sortType() == SortField.STRING_VAL)) {
executeTermResult = executeTermSortedStringTerm(request, indexFieldName, searcher, regexpPattern, fieldMapper, termDocs);
} else {
executeTermResult = executeTerms(request, indexFieldName, searcher, regexpPattern, fieldMapper, termDocs);
}
termDocs = executeTermResult.termDocs;
response.put(fieldName, executeTermResult.termsFreqs);
} catch (Exception e) {
// currently, just log
logger.warn("Failed to fetch terms for field [" + fieldName + "]", e);
}
}
return response;
} finally {
if (termDocs != null) {
try {
termDocs.close();
} catch (IOException e) {
// ignore
}
}
searcher.release();
}
}
static class ExecuteTermResult {
public TObjectIntHashMap<Object> termsFreqs;
public TermDocs termDocs;
ExecuteTermResult(TObjectIntHashMap<Object> termsFreqs, TermDocs termDocs) {
this.termsFreqs = termsFreqs;
this.termDocs = termDocs;
}
}
private ExecuteTermResult executeTerms(ShardTermsRequest request, String indexFieldName, Engine.Searcher searcher,
@Nullable Pattern regexpPattern, @Nullable FieldMapper fieldMapper, @Nullable TermDocs termDocs) throws IOException {
TObjectIntHashMap<Object> termsFreqs = new TObjectIntHashMap<Object>();
String sFrom = request.from();
if (sFrom == null) {
// really, only make sense for strings
sFrom = request.prefix();
}
Object from = sFrom;
if (from != null && fieldMapper != null) {
from = fieldMapper.valueFromString(sFrom);
}
String sTo = request.to();
Object to = sTo;
if (to != null && fieldMapper != null) {
to = fieldMapper.valueFromString(sTo);
}
TermEnum termEnum = null;
Comparator<TermFreq> comparator = request.sortType() == TermsRequest.SortType.TERM ? TermFreq.termComparator() : TermFreq.freqComparator();
BoundedTreeSet<TermFreq> sortedFreq = new BoundedTreeSet<TermFreq>(comparator, request.size());
try {
termEnum = searcher.reader().terms(new Term(indexFieldName, ""));
while (true) {
Term term = termEnum.term();
// have we reached the end?
if (term == null || indexFieldName != term.field()) { // StirngHelper.intern
break;
}
Object termValue = term.text();
if (fieldMapper != null) {
termValue = fieldMapper.valueFromTerm(term.text());
if (fieldMapper.shouldBreakTermEnumeration(termValue)) {
break;
}
if (termValue == null) {
continue;
}
}
// check on the from term
if (from != null) {
int fromCompareResult = ((Comparable) termValue).compareTo(from);
if (fromCompareResult < 0 || (fromCompareResult == 0 && !request.fromInclusive())) {
termEnum.next();
continue;
}
}
// does it match on the prefix?
if (request.prefix() != null && !term.text().startsWith(request.prefix())) {
break;
}
// does it match on regexp?
if (regexpPattern != null && !regexpPattern.matcher(term.text()).matches()) {
termEnum.next();
continue;
}
// check on the to term
if (to != null) {
int toCompareResult = ((Comparable) termValue).compareTo(to);
if (toCompareResult > 0 || (toCompareResult == 0 && !request.toInclusive())) {
break;
}
}
int docFreq = termEnum.docFreq();
if (request.exact()) {
if (termDocs == null) {
termDocs = searcher.reader().termDocs();
}
termDocs.seek(termEnum);
docFreq = 0;
while (termDocs.next()) {
if (!searcher.reader().isDeleted(termDocs.doc())) {
docFreq++;
}
}
}
sortedFreq.add(new TermFreq(termValue, docFreq));
if (!termEnum.next()) {
break;
}
}
} finally {
if (termEnum != null) {
try {
termEnum.close();
} catch (IOException e) {
// ignore
}
}
}
for (TermFreq termFreq : sortedFreq) {
termsFreqs.put(termFreq.term(), termFreq.docFreq());
}
return new ExecuteTermResult(termsFreqs, termDocs);
}
private ExecuteTermResult executeTermSortedStringTerm(ShardTermsRequest request, String indexFieldName, Engine.Searcher searcher,
@Nullable Pattern regexpPattern, @Nullable FieldMapper fieldMapper, @Nullable TermDocs termDocs) throws IOException {
TObjectIntHashMap<Object> termsFreqs = new TObjectIntHashMap<Object>();
String from = request.from();
if (from == null) {
from = request.prefix();
}
if (from == null) {
from = "";
}
Term fromTerm = new Term(indexFieldName, from);
String to = request.to();
if (to != null && fieldMapper != null) {
to = fieldMapper.indexedValue(to);
}
Term toTerm = to == null ? null : new Term(indexFieldName, to);
TermEnum termEnum = null;
try {
termEnum = searcher.reader().terms(fromTerm);
// skip the first if we are not inclusive on from
if (!request.fromInclusive() && request.from() != null) {
Term term = termEnum.term();
if (term != null && indexFieldName == term.field() && term.text().equals(request.from())) {
termEnum.next();
}
}
if (request.sortType() == TermsRequest.SortType.TERM) {
int counter = 0;
while (counter < request.size()) {
Term term = termEnum.term();
// have we reached the end?
if (term == null || indexFieldName != term.field()) { // StirngHelper.intern
break;
}
// convert to actual term text
if (fieldMapper != null) {
// valueAsString returns null indicating that this is not interesting
Object termObj = fieldMapper.valueFromTerm(term.text());
// if we need to break on this term enumeration, bail
if (fieldMapper.shouldBreakTermEnumeration(termObj)) {
break;
}
if (termObj == null) {
termEnum.next();
continue;
}
}
// does it match on the prefix?
if (request.prefix() != null && !term.text().startsWith(request.prefix())) {
break;
}
// does it match on regexp?
if (regexpPattern != null && !regexpPattern.matcher(term.text()).matches()) {
termEnum.next();
continue;
}
// check on the to term
if (toTerm != null) {
int toCompareResult = term.compareTo(toTerm);
if (toCompareResult > 0 || (toCompareResult == 0 && !request.toInclusive())) {
break;
}
}
int docFreq = termEnum.docFreq();
if (request.exact()) {
if (termDocs == null) {
termDocs = searcher.reader().termDocs();
}
termDocs.seek(termEnum);
docFreq = 0;
while (termDocs.next()) {
if (!searcher.reader().isDeleted(termDocs.doc())) {
docFreq++;
}
}
}
termsFreqs.put(term.text(), docFreq);
if (!termEnum.next()) {
break;
}
counter++;
}
}
} finally {
if (termEnum != null) {
try {
termEnum.close();
} catch (IOException e) {
// ignore
}
}
}
return new ExecuteTermResult(termsFreqs, termDocs);
}
@Override protected String transportAction() {
return TransportActions.TERMS;
}
@Override protected String transportShardAction() {
return "indices/terms/shard";
}
@Override protected TermsRequest newRequest() {
return new TermsRequest();
}
@Override protected ShardTermsRequest newShardRequest() {
return new ShardTermsRequest();
}
@Override protected ShardTermsRequest newShardRequest(ShardRouting shard, TermsRequest request) {
return new ShardTermsRequest(shard.index(), shard.id(), request);
}
@Override protected ShardTermsResponse newShardResponse() {
return new ShardTermsResponse();
}
@Override protected GroupShardsIterator shards(TermsRequest request, ClusterState clusterState) {
return indicesService.searchShards(clusterState, request.indices(), request.queryHint());
}
}

View File

@ -1,23 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
/**
* Terms action.
*/
package org.elasticsearch.action.terms;

View File

@ -35,8 +35,6 @@ import org.elasticsearch.action.mlt.MoreLikeThisRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.client.action.count.CountRequestBuilder;
import org.elasticsearch.client.action.delete.DeleteRequestBuilder;
import org.elasticsearch.client.action.deletebyquery.DeleteByQueryRequestBuilder;
@ -44,7 +42,6 @@ import org.elasticsearch.client.action.get.GetRequestBuilder;
import org.elasticsearch.client.action.index.IndexRequestBuilder;
import org.elasticsearch.client.action.search.SearchRequestBuilder;
import org.elasticsearch.client.action.search.SearchScrollRequestBuilder;
import org.elasticsearch.client.action.terms.TermsRequestBuilder;
import javax.annotation.Nullable;
@ -276,32 +273,6 @@ public interface Client {
*/
SearchScrollRequestBuilder prepareSearchScroll(String scrollId);
/**
* A terms request to get terms in one or more indices of specific fields and their
* document frequencies (in how many document each term exists).
*
* @param request The term request
* @return The result future
* @see Requests#termsRequest(String...)
*/
ActionFuture<TermsResponse> terms(TermsRequest request);
/**
* A terms request to get terms in one or more indices of specific fields and their
* document frequencies (in how many document each term exists).
*
* @param request The term request
* @param listener A listener to be notified of the result
* @see Requests#termsRequest(String...)
*/
void terms(TermsRequest request, ActionListener<TermsResponse> listener);
/**
* A terms request to get terms in one or more indices of specific fields and their
* document frequencies (in how many document each term exists).
*/
TermsRequestBuilder prepareTerms(String... indices);
/**
* A more like this action to search for documents that are "like" a specific document.
*

View File

@ -46,7 +46,6 @@ import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.mlt.MoreLikeThisRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.common.xcontent.XContentType;
/**
@ -125,18 +124,6 @@ public class Requests {
return new CountRequest(indices);
}
/**
* Terms request represent a request to get terms in one or more indices of specific fields and their
* document frequencies (in how many document each term exists).
*
* @param indices The indices the delete by query against. Use <tt>null</tt> or <tt>_all</tt> to execute against all indices
* @return The terms request
* @see org.elasticsearch.client.Client#terms(org.elasticsearch.action.terms.TermsRequest)
*/
public static TermsRequest termsRequest(String... indices) {
return new TermsRequest(indices);
}
/**
* More like this request represents a request to search for documents that are "like" the provided (fetched)
* document.

View File

@ -1,192 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client.action.terms;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.client.action.support.BaseRequestBuilder;
import org.elasticsearch.client.internal.InternalClient;
/**
* @author kimchy (shay.banon)
*/
public class TermsRequestBuilder extends BaseRequestBuilder<TermsRequest, TermsResponse> {
public TermsRequestBuilder(InternalClient client) {
super(client, new TermsRequest());
}
/**
* Sets the indices the terms will run against.
*/
public TermsRequestBuilder setIndices(String... indices) {
request.indices(indices);
return this;
}
/**
* The fields within each document which terms will be iterated over and returned with the
* document frequencies. By default will use the "_all" field.
*/
public TermsRequestBuilder setFields(String... fields) {
request.fields(fields);
return this;
}
/**
* The lower bound term from which the iteration will start. Defaults to start from the
* first.
*/
public TermsRequestBuilder setFrom(Object from) {
request.from(from);
return this;
}
/**
* Greater than (like setting from with fromIInclusive set to <tt>false</tt>).
*/
public TermsRequestBuilder setGreaterThan(Object from) {
request.gt(from);
return this;
}
/**
* Greater/equal than (like setting from with fromInclusive set to <tt>true</tt>).
*/
public TermsRequestBuilder setGreaterEqualsThan(Object from) {
request.gt(from);
return this;
}
/**
* Lower then (like setting to with toInclusive set to <tt>false</tt>)
*/
public TermsRequestBuilder setLowerThan(Object to) {
request.lt(to);
return this;
}
/**
* Lower/equal then (like setting to with toInclusive set to <tt>false</tt>)
*/
public TermsRequestBuilder setLowerEqualThan(Object to) {
request.lte(to);
return this;
}
/**
* Should the first from (if set using {@link #setFrom(Object)} be inclusive or not. Defaults
* to <tt>false</tt> (not inclusive / exclusive).
*/
public TermsRequestBuilder setFromInclusive(boolean fromInclusive) {
request.fromInclusive(fromInclusive);
return this;
}
/**
* The upper bound term to which the iteration will end. Defaults to unbound (<tt>null</tt>).
*/
public TermsRequestBuilder setTo(Object to) {
request.to(to);
return this;
}
/**
* Should the last to (if set using {@link #setTo(Object)} be inclusive or not. Defaults to
* <tt>true</tt>.
*/
public TermsRequestBuilder setToInclusive(boolean toInclusive) {
request.toInclusive(toInclusive);
return this;
}
/**
* An optional prefix from which the terms iteration will start (in lex order).
*/
public TermsRequestBuilder setPrefix(String prefix) {
request.prefix(prefix);
return this;
}
/**
* An optional regular expression to filter out terms (only the ones that match the regexp
* will return).
*/
public TermsRequestBuilder setRegexp(String regexp) {
request.regexp(regexp);
return this;
}
/**
* An optional minimum document frequency to filter out terms.
*/
public TermsRequestBuilder setMinFreq(int minFreq) {
request.minFreq(minFreq);
return this;
}
/**
* An optional maximum document frequency to filter out terms.
*/
public TermsRequestBuilder setMaxFreq(int maxFreq) {
request.maxFreq(maxFreq);
return this;
}
/**
* The number of term / doc freq pairs to return per field. Defaults to <tt>10</tt>.
*/
public TermsRequestBuilder setSize(int size) {
request.size(size);
return this;
}
/**
* The type of sorting for term / doc freq. Can either sort on term (lex) or doc frequency. Defaults to
* {@link TermsRequest.SortType#TERM}.
*/
public TermsRequestBuilder setSortType(TermsRequest.SortType sortType) {
request.sortType(sortType);
return this;
}
/**
* Sets the string representation of the sort type.
*/
public TermsRequestBuilder setSortType(String sortType) {
request.sortType(sortType);
return this;
}
/**
* Should the doc frequencies be exact frequencies. Exact frequencies takes into account deletes that
* have not been merged and cleaned (optimized). Note, when this is set to <tt>true</tt> this operation
* might be an expensive operation. Defaults to <tt>false</tt>.
*/
public TermsRequestBuilder setExact(boolean exact) {
request.exact(exact);
return this;
}
@Override protected void doExecute(ActionListener<TermsResponse> listener) {
client.terms(request, listener);
}
}

View File

@ -39,9 +39,6 @@ import org.elasticsearch.action.index.TransportIndexAction;
import org.elasticsearch.action.mlt.MoreLikeThisRequest;
import org.elasticsearch.action.mlt.TransportMoreLikeThisAction;
import org.elasticsearch.action.search.*;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.action.terms.TransportTermsAction;
import org.elasticsearch.client.AdminClient;
import org.elasticsearch.client.internal.InternalClient;
import org.elasticsearch.client.support.AbstractClient;
@ -72,15 +69,13 @@ public class NodeClient extends AbstractClient implements InternalClient {
private final TransportSearchScrollAction searchScrollAction;
private final TransportTermsAction termsAction;
private final TransportMoreLikeThisAction moreLikeThisAction;
@Inject public NodeClient(Settings settings, ThreadPool threadPool, NodeAdminClient admin,
TransportIndexAction indexAction, TransportDeleteAction deleteAction,
TransportDeleteByQueryAction deleteByQueryAction, TransportGetAction getAction, TransportCountAction countAction,
TransportSearchAction searchAction, TransportSearchScrollAction searchScrollAction,
TransportTermsAction termsAction, TransportMoreLikeThisAction moreLikeThisAction) {
TransportMoreLikeThisAction moreLikeThisAction) {
this.threadPool = threadPool;
this.admin = admin;
this.indexAction = indexAction;
@ -90,7 +85,6 @@ public class NodeClient extends AbstractClient implements InternalClient {
this.countAction = countAction;
this.searchAction = searchAction;
this.searchScrollAction = searchScrollAction;
this.termsAction = termsAction;
this.moreLikeThisAction = moreLikeThisAction;
}
@ -162,14 +156,6 @@ public class NodeClient extends AbstractClient implements InternalClient {
searchScrollAction.execute(request, listener);
}
@Override public ActionFuture<TermsResponse> terms(TermsRequest request) {
return termsAction.execute(request);
}
@Override public void terms(TermsRequest request, ActionListener<TermsResponse> listener) {
termsAction.execute(request, listener);
}
@Override public ActionFuture<SearchResponse> moreLikeThis(MoreLikeThisRequest request) {
return moreLikeThisAction.execute(request);
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.client.action.get.GetRequestBuilder;
import org.elasticsearch.client.action.index.IndexRequestBuilder;
import org.elasticsearch.client.action.search.SearchRequestBuilder;
import org.elasticsearch.client.action.search.SearchScrollRequestBuilder;
import org.elasticsearch.client.action.terms.TermsRequestBuilder;
import org.elasticsearch.client.internal.InternalClient;
import javax.annotation.Nullable;
@ -79,8 +78,4 @@ public abstract class AbstractClient implements InternalClient {
@Override public CountRequestBuilder prepareCount(String... indices) {
return new CountRequestBuilder(this).setIndices(indices);
}
@Override public TermsRequestBuilder prepareTerms(String... indices) {
return new TermsRequestBuilder(this).setIndices(indices);
}
}

View File

@ -36,8 +36,6 @@ import org.elasticsearch.action.mlt.MoreLikeThisRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.client.AdminClient;
import org.elasticsearch.client.support.AbstractClient;
import org.elasticsearch.client.transport.action.ClientTransportActionModule;
@ -281,14 +279,6 @@ public class TransportClient extends AbstractClient {
internalClient.searchScroll(request, listener);
}
@Override public ActionFuture<TermsResponse> terms(TermsRequest request) {
return internalClient.terms(request);
}
@Override public void terms(TermsRequest request, ActionListener<TermsResponse> listener) {
internalClient.terms(request, listener);
}
@Override public ActionFuture<SearchResponse> moreLikeThis(MoreLikeThisRequest request) {
return internalClient.moreLikeThis(request);
}

View File

@ -45,7 +45,6 @@ import org.elasticsearch.client.transport.action.get.ClientTransportGetAction;
import org.elasticsearch.client.transport.action.index.ClientTransportIndexAction;
import org.elasticsearch.client.transport.action.search.ClientTransportSearchAction;
import org.elasticsearch.client.transport.action.search.ClientTransportSearchScrollAction;
import org.elasticsearch.client.transport.action.terms.ClientTransportTermsAction;
import org.elasticsearch.common.inject.AbstractModule;
/**
@ -61,7 +60,6 @@ public class ClientTransportActionModule extends AbstractModule {
bind(ClientTransportCountAction.class).asEagerSingleton();
bind(ClientTransportSearchAction.class).asEagerSingleton();
bind(ClientTransportSearchScrollAction.class).asEagerSingleton();
bind(ClientTransportTermsAction.class).asEagerSingleton();
bind(ClientTransportIndicesStatusAction.class).asEagerSingleton();
bind(ClientTransportRefreshAction.class).asEagerSingleton();

View File

@ -1,42 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.client.transport.action.terms;
import org.elasticsearch.action.TransportActions;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.client.transport.action.support.BaseClientTransportAction;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.transport.TransportService;
/**
* @author kimchy (Shay Banon)
*/
public class ClientTransportTermsAction extends BaseClientTransportAction<TermsRequest, TermsResponse> {
@Inject public ClientTransportTermsAction(Settings settings, TransportService transportService) {
super(settings, transportService, TermsResponse.class);
}
@Override protected String action() {
return TransportActions.TERMS;
}
}

View File

@ -36,8 +36,6 @@ import org.elasticsearch.action.mlt.MoreLikeThisRequest;
import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.client.AdminClient;
import org.elasticsearch.client.internal.InternalClient;
import org.elasticsearch.client.support.AbstractClient;
@ -50,7 +48,6 @@ import org.elasticsearch.client.transport.action.index.ClientTransportIndexActio
import org.elasticsearch.client.transport.action.mlt.ClientTransportMoreLikeThisAction;
import org.elasticsearch.client.transport.action.search.ClientTransportSearchAction;
import org.elasticsearch.client.transport.action.search.ClientTransportSearchScrollAction;
import org.elasticsearch.client.transport.action.terms.ClientTransportTermsAction;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -81,8 +78,6 @@ public class InternalTransportClient extends AbstractClient implements InternalC
private final ClientTransportSearchScrollAction searchScrollAction;
private final ClientTransportTermsAction termsAction;
private final ClientTransportMoreLikeThisAction moreLikeThisAction;
@Inject public InternalTransportClient(Settings settings, ThreadPool threadPool,
@ -90,7 +85,7 @@ public class InternalTransportClient extends AbstractClient implements InternalC
ClientTransportIndexAction indexAction, ClientTransportDeleteAction deleteAction, ClientTransportGetAction getAction,
ClientTransportDeleteByQueryAction deleteByQueryAction, ClientTransportCountAction countAction,
ClientTransportSearchAction searchAction, ClientTransportSearchScrollAction searchScrollAction,
ClientTransportTermsAction termsAction, ClientTransportMoreLikeThisAction moreLikeThisAction) {
ClientTransportMoreLikeThisAction moreLikeThisAction) {
this.threadPool = threadPool;
this.nodesService = nodesService;
this.adminClient = adminClient;
@ -102,7 +97,6 @@ public class InternalTransportClient extends AbstractClient implements InternalC
this.countAction = countAction;
this.searchAction = searchAction;
this.searchScrollAction = searchScrollAction;
this.termsAction = termsAction;
this.moreLikeThisAction = moreLikeThisAction;
}
@ -237,23 +231,6 @@ public class InternalTransportClient extends AbstractClient implements InternalC
});
}
@Override public ActionFuture<TermsResponse> terms(final TermsRequest request) {
return nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<TermsResponse>>() {
@Override public ActionFuture<TermsResponse> doWithNode(DiscoveryNode node) throws ElasticSearchException {
return termsAction.execute(node, request);
}
});
}
@Override public void terms(final TermsRequest request, final ActionListener<TermsResponse> listener) {
nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<Void>>() {
@Override public ActionFuture<Void> doWithNode(DiscoveryNode node) throws ElasticSearchException {
termsAction.execute(node, request, listener);
return null;
}
});
}
@Override public ActionFuture<SearchResponse> moreLikeThis(final MoreLikeThisRequest request) {
return nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<SearchResponse>>() {
@Override public ActionFuture<SearchResponse> doWithNode(DiscoveryNode node) throws ElasticSearchException {

View File

@ -118,11 +118,6 @@ public interface FieldMapper<T> {
*/
Object valueForSearch(Fieldable field);
/**
* Returns the value that will be returned to the user (similar to {@link #valueForSearch(org.apache.lucene.document.Fieldable)}).
*/
Object valueForSearch(Object value);
/**
* Returns the actual value of the field.
*/
@ -139,18 +134,6 @@ public interface FieldMapper<T> {
*/
Object valueFromTerm(String term);
/**
* Parses a string that represents the field into its value. For example, with numbers,
* it parses "1" to 1.
*/
Object valueFromString(String text);
/**
* Return <tt>true</tt> if this term value indicates breaking out of term enumeration on this
* field. The term text passed is the one returned from {@link #valueFromTerm(String)}.
*/
boolean shouldBreakTermEnumeration(Object text);
/**
* Returns the indexed value.
*/

View File

@ -115,10 +115,6 @@ public class XContentBoostFieldMapper extends XContentNumberFieldMapper<Float> i
return NumericUtils.prefixCodedToFloat(term);
}
@Override public Object valueFromString(String text) {
return Float.parseFloat(text);
}
@Override public Query rangeQuery(String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newFloatRange(names.indexName(), precisionStep,
lowerTerm == null ? null : Float.parseFloat(lowerTerm),

View File

@ -133,10 +133,6 @@ public class XContentDateFieldMapper extends XContentNumberFieldMapper<Long> {
return valueAsString(field);
}
@Override public Object valueForSearch(Object value) {
return dateTimeFormatter.printer().print((Long) value);
}
@Override public String valueAsString(Fieldable field) {
Long value = value(field);
if (value == null) {
@ -157,10 +153,6 @@ public class XContentDateFieldMapper extends XContentNumberFieldMapper<Long> {
return NumericUtils.prefixCodedToLong(term);
}
@Override public Object valueFromString(String text) {
return dateTimeFormatter.parser().parseMillis(text);
}
@Override public Query rangeQuery(String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
lowerTerm == null ? null : dateTimeFormatter.parser().parseMillis(lowerTerm),

View File

@ -126,10 +126,6 @@ public class XContentDoubleFieldMapper extends XContentNumberFieldMapper<Double>
return NumericUtils.prefixCodedToDouble(term);
}
@Override public Object valueFromString(String text) {
return Double.parseDouble(text);
}
@Override public Query rangeQuery(String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newDoubleRange(names.indexName(), precisionStep,
lowerTerm == null ? null : Double.parseDouble(lowerTerm),

View File

@ -299,10 +299,6 @@ public abstract class XContentFieldMapper<T> implements FieldMapper<T>, XContent
return valueAsString(field);
}
@Override public Object valueForSearch(Object value) {
return value;
}
/**
* Simply returns the same string.
*/
@ -310,17 +306,6 @@ public abstract class XContentFieldMapper<T> implements FieldMapper<T>, XContent
return term;
}
@Override public Object valueFromString(String text) {
return text;
}
/**
* Never break on this term enumeration value.
*/
@Override public boolean shouldBreakTermEnumeration(Object text) {
return false;
}
@Override public String indexedValue(String value) {
return value;
}

View File

@ -125,10 +125,6 @@ public class XContentFloatFieldMapper extends XContentNumberFieldMapper<Float> {
return NumericUtils.prefixCodedToFloat(term);
}
@Override public Object valueFromString(String text) {
return Float.parseFloat(text);
}
@Override public Query rangeQuery(String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newFloatRange(names.indexName(), precisionStep,
lowerTerm == null ? null : Float.parseFloat(lowerTerm),

View File

@ -125,10 +125,6 @@ public class XContentIntegerFieldMapper extends XContentNumberFieldMapper<Intege
return NumericUtils.prefixCodedToInt(term);
}
@Override public Object valueFromString(String text) {
return Integer.parseInt(text);
}
@Override public Query rangeQuery(String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
lowerTerm == null ? null : Integer.parseInt(lowerTerm),

View File

@ -125,10 +125,6 @@ public class XContentLongFieldMapper extends XContentNumberFieldMapper<Long> {
return NumericUtils.prefixCodedToLong(term);
}
@Override public Object valueFromString(String text) {
return Long.parseLong(text);
}
@Override public Query rangeQuery(String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newLongRange(names.indexName(), precisionStep,
lowerTerm == null ? null : Long.parseLong(lowerTerm),

View File

@ -156,15 +156,6 @@ public abstract class XContentNumberFieldMapper<T extends Number> extends XConte
@Override public abstract Object valueFromTerm(String term);
@Override public abstract Object valueFromString(String text);
/**
* Breaks on this text if its <tt>null</tt>.
*/
@Override public boolean shouldBreakTermEnumeration(Object text) {
return text == null;
}
@Override protected void doXContentBody(XContentBuilder builder) throws IOException {
super.doXContentBody(builder);
builder.field("precision_step", precisionStep);

View File

@ -125,10 +125,6 @@ public class XContentShortFieldMapper extends XContentNumberFieldMapper<Short> {
return NumericUtils.prefixCodedToInt(term);
}
@Override public Object valueFromString(String text) {
return Short.parseShort(text);
}
@Override public Query rangeQuery(String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
return NumericRangeQuery.newIntRange(names.indexName(), precisionStep,
lowerTerm == null ? null : Integer.parseInt(lowerTerm),

View File

@ -49,7 +49,6 @@ import org.elasticsearch.rest.action.main.RestMainAction;
import org.elasticsearch.rest.action.mlt.RestMoreLikeThisAction;
import org.elasticsearch.rest.action.search.RestSearchAction;
import org.elasticsearch.rest.action.search.RestSearchScrollAction;
import org.elasticsearch.rest.action.terms.RestTermsAction;
/**
* @author kimchy (Shay Banon)
@ -94,7 +93,6 @@ public class RestActionModule extends AbstractModule {
bind(RestDeleteByQueryAction.class).asEagerSingleton();
bind(RestCountAction.class).asEagerSingleton();
bind(RestTermsAction.class).asEagerSingleton();
bind(RestSearchAction.class).asEagerSingleton();
bind(RestSearchScrollAction.class).asEagerSingleton();

View File

@ -1,177 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.rest.action.terms;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.broadcast.BroadcastOperationThreading;
import org.elasticsearch.action.terms.FieldTermsFreq;
import org.elasticsearch.action.terms.TermFreq;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.builder.XContentBuilder;
import org.elasticsearch.rest.*;
import org.elasticsearch.rest.action.support.RestXContentBuilder;
import java.io.IOException;
import java.util.regex.Pattern;
import static org.elasticsearch.rest.RestResponse.Status.*;
import static org.elasticsearch.rest.action.support.RestActions.*;
/**
* @author kimchy (shay.banon)
*/
public class RestTermsAction extends BaseRestHandler {
private final static Pattern fieldsPattern;
static {
fieldsPattern = Pattern.compile(",");
}
@Inject public RestTermsAction(Settings settings, Client client, RestController controller) {
super(settings, client);
controller.registerHandler(RestRequest.Method.POST, "/_terms", this);
controller.registerHandler(RestRequest.Method.GET, "/_terms", this);
controller.registerHandler(RestRequest.Method.POST, "/{index}/_terms", this);
controller.registerHandler(RestRequest.Method.GET, "/{index}/_terms", this);
}
@Override public void handleRequest(final RestRequest request, final RestChannel channel) {
TermsRequest termsRequest = new TermsRequest(splitIndices(request.param("index")));
// we just send back a response, no need to fork a listener
termsRequest.listenerThreaded(false);
try {
BroadcastOperationThreading operationThreading = BroadcastOperationThreading.fromString(request.param("operation_threading"), BroadcastOperationThreading.SINGLE_THREAD);
if (operationThreading == BroadcastOperationThreading.NO_THREADS) {
// since we don't spawn, don't allow no_threads, but change it to a single thread
operationThreading = BroadcastOperationThreading.SINGLE_THREAD;
}
termsRequest.operationThreading(operationThreading);
String sField = request.param("fields");
if (sField != null) {
String[] sFields = fieldsPattern.split(sField);
termsRequest.fields(sFields);
}
termsRequest.from(request.param("from"));
termsRequest.to(request.param("to"));
termsRequest.fromInclusive(request.paramAsBoolean("from_inclusive", termsRequest.fromInclusive()));
termsRequest.toInclusive(request.paramAsBoolean("to_inclusive", termsRequest.toInclusive()));
Object temp = request.param("gt");
if (temp != null) {
termsRequest.gt(temp);
} else {
temp = request.param("gte");
if (temp != null) {
termsRequest.gte(temp);
}
}
temp = request.param("lt");
if (temp != null) {
termsRequest.lt(temp);
} else {
temp = request.param("lte");
if (temp != null) {
termsRequest.lte(temp);
}
}
termsRequest.exact(request.paramAsBoolean("exact", termsRequest.exact()));
termsRequest.minFreq(request.paramAsInt("min_freq", termsRequest.minFreq()));
termsRequest.maxFreq(request.paramAsInt("max_freq", termsRequest.maxFreq()));
termsRequest.size(request.paramAsInt("size", termsRequest.size()));
termsRequest.prefix(request.param("prefix"));
termsRequest.regexp(request.param("regexp"));
termsRequest.sortType(request.param("sort"));
} catch (Exception e) {
try {
XContentBuilder builder = RestXContentBuilder.restContentBuilder(request);
channel.sendResponse(new XContentRestResponse(request, BAD_REQUEST, builder.startObject().field("error", e.getMessage()).endObject()));
} catch (IOException e1) {
logger.error("Failed to send failure response", e1);
}
return;
}
final boolean termsAsArray = request.paramAsBoolean("terms_as_array", true);
client.terms(termsRequest, new ActionListener<TermsResponse>() {
@Override public void onResponse(TermsResponse response) {
try {
XContentBuilder builder = RestXContentBuilder.restContentBuilder(request);
builder.startObject();
buildBroadcastShardsHeader(builder, response);
builder.startObject("docs");
builder.field("num_docs", response.numDocs());
builder.field("max_doc", response.maxDoc());
builder.field("deleted_docs", response.deletedDocs());
builder.endObject();
builder.startObject("fields");
for (FieldTermsFreq fieldTermsFreq : response.fields()) {
builder.startObject(fieldTermsFreq.fieldName());
if (!termsAsArray) {
builder.startObject("terms");
for (TermFreq termFreq : fieldTermsFreq.termsFreqs()) {
builder.startObject(termFreq.termAsString());
builder.field("doc_freq", termFreq.docFreq());
builder.endObject();
}
builder.endObject();
} else {
builder.startArray("terms");
for (TermFreq termFreq : fieldTermsFreq.termsFreqs()) {
builder.startObject();
builder.field("term", termFreq.term());
builder.field("doc_freq", termFreq.docFreq());
builder.endObject();
}
builder.endArray();
}
builder.endObject();
}
builder.endObject();
builder.endObject();
channel.sendResponse(new XContentRestResponse(request, OK, builder));
} catch (Exception e) {
onFailure(e);
}
}
@Override public void onFailure(Throwable e) {
try {
channel.sendResponse(new XContentThrowableRestResponse(request, e));
} catch (IOException e1) {
logger.error("Failed to send failure response", e1);
}
}
});
}
}

View File

@ -1,41 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.test.integration.terms;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.transport.TransportClient;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.node.internal.InternalNode;
import org.elasticsearch.transport.TransportService;
import org.testng.annotations.Test;
/**
* @author kimchy (shay.banon)
*/
@Test
public class ClientTransportTermsActionTests extends TermsActionTests {
@Override protected Client getClient() {
TransportAddress server1Address = ((InternalNode) node("server1")).injector().getInstance(TransportService.class).boundAddress().publishAddress();
TransportClient client = new TransportClient();
client.addTransportAddress(server1Address);
return client;
}
}

View File

@ -1,6 +0,0 @@
cluster:
routing:
schedule: 100ms
index:
number_of_shards: 5
number_of_replicas: 1

View File

@ -1,468 +0,0 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.test.integration.terms;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.indices.optimize.OptimizeResponse;
import org.elasticsearch.action.admin.indices.status.IndexStatus;
import org.elasticsearch.action.terms.TermsRequest;
import org.elasticsearch.action.terms.TermsResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.test.integration.AbstractNodesTests;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import static org.elasticsearch.action.terms.TermsRequest.SortType.*;
import static org.elasticsearch.client.Requests.*;
import static org.elasticsearch.common.collect.MapBuilder.*;
import static org.elasticsearch.common.xcontent.XContentFactory.*;
import static org.hamcrest.MatcherAssert.*;
import static org.hamcrest.Matchers.*;
/**
* @author kimchy (shay.banon)
*/
@Test
public class TermsActionTests extends AbstractNodesTests {
private Client client;
@BeforeMethod public void createNodesAndClient() throws Exception {
startNode("server1");
startNode("server2");
client = getClient();
logger.info("Creating index test");
client.admin().indices().create(createIndexRequest("test")).actionGet();
logger.info("Running Cluster Health");
ClusterHealthResponse clusterHealth = client.admin().cluster().health(clusterHealth().waitForGreenStatus()).actionGet();
logger.info("Done Cluster Health, status " + clusterHealth.status());
assertThat(clusterHealth.timedOut(), equalTo(false));
assertThat(clusterHealth.status(), equalTo(ClusterHealthStatus.GREEN));
}
@AfterMethod public void closeNodes() {
client.close();
closeAllNodes();
}
protected Client getClient() {
return client("server2");
}
@Test public void testSimpleStringTerms() throws Exception {
IndexStatus indexStatus = client.admin().indices().prepareStatus("test").execute().actionGet().index("test");
// verify no freqs
logger.info("Verify no freqs");
TermsResponse termsResponse = client.prepareTerms("test").setFields("value").execute().actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat("no term freqs for the 'value' since nothing is indexed", termsResponse.field("value").iterator().hasNext(), equalTo(false));
logger.info("Index [1]");
client.index(indexRequest("test").type("type1").id("1").source(jsonBuilder().startObject().field("value", "aaa").endObject())).actionGet();
logger.info("Refresh");
client.admin().indices().refresh(refreshRequest()).actionGet();
logger.info("Verify freqs");
termsResponse = client.terms(termsRequest("test").fields("value")).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.numDocs(), equalTo(1l));
assertThat(termsResponse.maxDoc(), equalTo(1l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(-1));
logger.info("Index [2]");
client.index(indexRequest("test").type("type1").id("2").source(jsonBuilder().startObject().field("value", "bbb bbb").endObject())).actionGet();
logger.info("Refresh");
client.admin().indices().refresh(refreshRequest()).actionGet();
logger.info("Verify freqs");
termsResponse = client.terms(termsRequest("test").fields("value")).actionGet();
assertThat(termsResponse.numDocs(), equalTo(2l));
assertThat(termsResponse.maxDoc(), equalTo(2l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(1));
logger.info("Verify freqs (no fields, on _all)");
termsResponse = client.terms(termsRequest("test")).actionGet();
assertThat(termsResponse.numDocs(), equalTo(2l));
assertThat(termsResponse.maxDoc(), equalTo(2l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("_all").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("_all").docFreq("bbb"), equalTo(1));
logger.info("Delete 3");
client.index(indexRequest("test").type("type1").id("3").source(jsonBuilder().startObject().field("value", "bbb").endObject())).actionGet();
logger.info("Refresh");
client.admin().indices().refresh(refreshRequest()).actionGet();
logger.info("Verify freqs");
termsResponse = client.terms(termsRequest("test").fields("value")).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(3l));
assertThat(termsResponse.maxDoc(), equalTo(3l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(2));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(2));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("aaa"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
assertThat(termsResponse.field("value").termsFreqs()[1].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[1].docFreq(), equalTo(2));
logger.info("Verify freqs (sort gy freq)");
termsResponse = client.terms(termsRequest("test").fields("value").sortType(FREQ)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(2));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(2));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(2));
assertThat(termsResponse.field("value").termsFreqs()[1].termAsString(), equalTo("aaa"));
assertThat(termsResponse.field("value").termsFreqs()[1].docFreq(), equalTo(1));
logger.info("Verify freq (size and sort by freq)");
termsResponse = client.terms(termsRequest("test").fields("value").sortType(FREQ).size(1)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(-1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(2));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(1));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(2));
logger.info("Verify freq (minFreq with sort by freq)");
termsResponse = client.terms(termsRequest("test").fields("value").sortType(FREQ).minFreq(2)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(-1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(2));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(1));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(2));
logger.info("Verify freq (prefix with sort by freq)");
termsResponse = client.terms(termsRequest("test").fields("value").sortType(FREQ).prefix("bb")).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(-1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(2));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(1));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(2));
// test deleting the last doc
logger.info("Delete [3]");
client.delete(deleteRequest("test").type("type1").id("3")).actionGet();
logger.info("Refresh");
client.admin().indices().refresh(refreshRequest()).actionGet();
logger.info("Verify freq (even after refresh, won't see the delete)");
termsResponse = client.terms(termsRequest("test").fields("value")).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(2l));
assertThat(termsResponse.maxDoc(), equalTo(3l));
assertThat(termsResponse.deletedDocs(), equalTo(1l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(2));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(2));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("aaa"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
assertThat(termsResponse.field("value").termsFreqs()[1].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[1].docFreq(), equalTo(2));
logger.info("Verify freq (with exact, should see the delete)");
termsResponse = client.terms(termsRequest("test").fields("value").exact(true)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(1));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(2));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("aaa"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
assertThat(termsResponse.field("value").termsFreqs()[1].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[1].docFreq(), equalTo(1));
logger.info("Optimize (onlyExpungeDeletes with refresh)");
OptimizeResponse optimizeResponse = client.admin().indices().optimize(optimizeRequest("test").onlyExpungeDeletes(true).refresh(true)).actionGet();
logger.info("Verify freq (we will see the delete now, without exact)");
termsResponse = client.terms(termsRequest("test").fields("value").exact(false)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(2l));
assertThat(termsResponse.maxDoc(), equalTo(2l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("value").docFreq("aaa"), equalTo(1));
assertThat(termsResponse.field("value").docFreq("bbb"), equalTo(1));
// check the order
assertThat(termsResponse.field("value").termsFreqs().length, equalTo(2));
assertThat(termsResponse.field("value").termsFreqs()[0].termAsString(), equalTo("aaa"));
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
assertThat(termsResponse.field("value").termsFreqs()[1].termAsString(), equalTo("bbb"));
assertThat(termsResponse.field("value").termsFreqs()[1].docFreq(), equalTo(1));
}
@Test public void testNumberedTerms() throws Exception {
IndexStatus indexStatus = client.admin().indices().status(indicesStatus("test")).actionGet().index("test");
logger.info("Index ...");
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 1).put("fl", 2.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 1).put("fl", 2.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 1).put("fl", 2.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 1).put("fl", 2.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 2).put("fl", 3.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 2).put("fl", 3.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 2).put("fl", 3.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 3).put("fl", 4.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 11).put("fl", 12.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 11).put("fl", 12.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 12).put("fl", 13.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 12).put("fl", 13.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 12).put("fl", 13.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 13).put("fl", 14.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 13).put("fl", 14.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 21).put("fl", 20.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 22).put("fl", 21.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 22).put("fl", 21.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 22).put("fl", 21.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 22).put("fl", 21.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 22).put("fl", 21.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 22).put("fl", 21.0f).map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("int", 23).put("fl", 22.0f).map())).actionGet();
logger.info("Refresh");
client.admin().indices().refresh(refreshRequest()).actionGet();
logger.info("Verify int with sort on term");
TermsResponse termsResponse = client.terms(termsRequest("test").fields("int").sortType(TermsRequest.SortType.TERM)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(23l));
assertThat(termsResponse.maxDoc(), equalTo(23l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("int").docFreq(1), equalTo(4));
assertThat(termsResponse.field("int").docFreq(2), equalTo(3));
// check the order
assertThat(termsResponse.field("int").termsFreqs().length, equalTo(9));
assertThat(termsResponse.field("int").termsFreqs()[0].termAsString(), equalTo("1"));
assertThat(termsResponse.field("int").termsFreqs()[0].docFreq(), equalTo(4));
assertThat(termsResponse.field("int").termsFreqs()[1].termAsString(), equalTo("2"));
assertThat(termsResponse.field("int").termsFreqs()[1].docFreq(), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs()[2].termAsString(), equalTo("3"));
assertThat(termsResponse.field("int").termsFreqs()[2].docFreq(), equalTo(1));
assertThat(termsResponse.field("int").termsFreqs()[3].termAsString(), equalTo("11"));
assertThat(termsResponse.field("int").termsFreqs()[3].docFreq(), equalTo(2));
assertThat(termsResponse.field("int").termsFreqs()[4].termAsString(), equalTo("12"));
assertThat(termsResponse.field("int").termsFreqs()[4].docFreq(), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs()[5].termAsString(), equalTo("13"));
assertThat(termsResponse.field("int").termsFreqs()[5].docFreq(), equalTo(2));
assertThat(termsResponse.field("int").termsFreqs()[6].termAsString(), equalTo("21"));
assertThat(termsResponse.field("int").termsFreqs()[6].docFreq(), equalTo(1));
logger.info("Verify int with sort on freq");
termsResponse = client.terms(termsRequest("test").fields("int").sortType(TermsRequest.SortType.FREQ)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(23l));
assertThat(termsResponse.maxDoc(), equalTo(23l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("int").docFreq(1), equalTo(4));
assertThat(termsResponse.field("int").docFreq(2), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs().length, equalTo(9));
assertThat(termsResponse.field("int").termsFreqs()[0].termAsString(), equalTo("22"));
assertThat(termsResponse.field("int").termsFreqs()[0].docFreq(), equalTo(6));
assertThat(termsResponse.field("int").termsFreqs()[1].termAsString(), equalTo("1"));
assertThat(termsResponse.field("int").termsFreqs()[1].docFreq(), equalTo(4));
logger.info("Verify int with sort on freq and from 2 to 11");
termsResponse = client.terms(termsRequest("test").fields("int").sortType(TermsRequest.SortType.FREQ).from(2).to(11)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(23l));
assertThat(termsResponse.maxDoc(), equalTo(23l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("int").docFreq(1), equalTo(-1));
assertThat(termsResponse.field("int").docFreq(2), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs().length, equalTo(3));
assertThat(termsResponse.field("int").termsFreqs()[0].termAsString(), equalTo("2"));
assertThat(termsResponse.field("int").termsFreqs()[0].docFreq(), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs()[1].termAsString(), equalTo("11"));
assertThat(termsResponse.field("int").termsFreqs()[1].docFreq(), equalTo(2));
assertThat(termsResponse.field("int").termsFreqs()[2].termAsString(), equalTo("3"));
assertThat(termsResponse.field("int").termsFreqs()[2].docFreq(), equalTo(1));
logger.info("Verify int with sort on term and from 2 to 11");
termsResponse = client.terms(termsRequest("test").fields("int").sortType(TermsRequest.SortType.TERM).from(2).to(11)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(23l));
assertThat(termsResponse.maxDoc(), equalTo(23l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("int").docFreq(1), equalTo(-1));
assertThat(termsResponse.field("int").docFreq(2), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs().length, equalTo(3));
assertThat(termsResponse.field("int").termsFreqs()[0].termAsString(), equalTo("2"));
assertThat(termsResponse.field("int").termsFreqs()[0].docFreq(), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs()[1].termAsString(), equalTo("3"));
assertThat(termsResponse.field("int").termsFreqs()[1].docFreq(), equalTo(1));
assertThat(termsResponse.field("int").termsFreqs()[2].termAsString(), equalTo("11"));
assertThat(termsResponse.field("int").termsFreqs()[2].docFreq(), equalTo(2));
logger.info("Verify int with sort on term and from 2 to 11, fromInclusive=false");
termsResponse = client.terms(termsRequest("test").fields("int").sortType(TermsRequest.SortType.TERM).from(2).to(11).fromInclusive(false)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(23l));
assertThat(termsResponse.maxDoc(), equalTo(23l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("int").docFreq(1), equalTo(-1));
assertThat(termsResponse.field("int").docFreq(3), equalTo(1));
assertThat(termsResponse.field("int").termsFreqs().length, equalTo(2));
assertThat(termsResponse.field("int").termsFreqs()[0].termAsString(), equalTo("3"));
assertThat(termsResponse.field("int").termsFreqs()[0].docFreq(), equalTo(1));
assertThat(termsResponse.field("int").termsFreqs()[1].termAsString(), equalTo("11"));
assertThat(termsResponse.field("int").termsFreqs()[1].docFreq(), equalTo(2));
logger.info("Verify int with sort on term and from 2 to 11, toInclusive=false");
termsResponse = client.terms(termsRequest("test").fields("int").sortType(TermsRequest.SortType.TERM).from(2).to(11).toInclusive(false)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(23l));
assertThat(termsResponse.maxDoc(), equalTo(23l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("int").docFreq(1), equalTo(-1));
assertThat(termsResponse.field("int").docFreq(2), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs().length, equalTo(2));
assertThat(termsResponse.field("int").termsFreqs()[0].termAsString(), equalTo("2"));
assertThat(termsResponse.field("int").termsFreqs()[0].docFreq(), equalTo(3));
assertThat(termsResponse.field("int").termsFreqs()[1].termAsString(), equalTo("3"));
assertThat(termsResponse.field("int").termsFreqs()[1].docFreq(), equalTo(1));
}
@Test public void testDateTerms() throws Exception {
IndexStatus indexStatus = client.admin().indices().status(indicesStatus("test")).actionGet().index("test");
logger.info("Index ...");
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("date", "2003-01-01").map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("date", "2003-01-01").map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("date", "2003-01-02").map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("date", "2003-01-03").map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("date", "2003-01-03").map())).actionGet();
client.index(indexRequest("test").type("type1").source(newMapBuilder().put("date", "2003-01-03").map())).actionGet();
logger.info("Refresh");
client.admin().indices().refresh(refreshRequest()).actionGet();
logger.info("Verify int with sort on term");
TermsResponse termsResponse = client.terms(termsRequest("test").fields("date").sortType(TermsRequest.SortType.TERM)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(6l));
assertThat(termsResponse.maxDoc(), equalTo(6l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("date").docFreq("2003-01-01T00:00:00.000Z"), equalTo(2));
assertThat(termsResponse.field("date").docFreq("2003-01-02T00:00:00.000Z"), equalTo(1));
assertThat(termsResponse.field("date").docFreq("2003-01-03T00:00:00.000Z"), equalTo(3));
assertThat(termsResponse.field("date").termsFreqs().length, equalTo(3));
assertThat(termsResponse.field("date").termsFreqs()[0].termAsString(), equalTo("2003-01-01T00:00:00.000Z"));
assertThat(termsResponse.field("date").termsFreqs()[0].docFreq(), equalTo(2));
assertThat(termsResponse.field("date").termsFreqs()[1].termAsString(), equalTo("2003-01-02T00:00:00.000Z"));
assertThat(termsResponse.field("date").termsFreqs()[1].docFreq(), equalTo(1));
assertThat(termsResponse.field("date").termsFreqs()[2].termAsString(), equalTo("2003-01-03T00:00:00.000Z"));
assertThat(termsResponse.field("date").termsFreqs()[2].docFreq(), equalTo(3));
logger.info("Verify int with sort on freq");
termsResponse = client.terms(termsRequest("test").fields("date").sortType(TermsRequest.SortType.FREQ)).actionGet();
assertThat(termsResponse.successfulShards(), equalTo(indexStatus.shards().size()));
assertThat(termsResponse.failedShards(), equalTo(0));
assertThat(termsResponse.numDocs(), equalTo(6l));
assertThat(termsResponse.maxDoc(), equalTo(6l));
assertThat(termsResponse.deletedDocs(), equalTo(0l));
assertThat(termsResponse.fieldsAsMap().isEmpty(), equalTo(false));
assertThat(termsResponse.field("date").docFreq("2003-01-01T00:00:00.000Z"), equalTo(2));
assertThat(termsResponse.field("date").docFreq("2003-01-02T00:00:00.000Z"), equalTo(1));
assertThat(termsResponse.field("date").docFreq("2003-01-03T00:00:00.000Z"), equalTo(3));
assertThat(termsResponse.field("date").termsFreqs().length, equalTo(3));
assertThat(termsResponse.field("date").termsFreqs()[0].termAsString(), equalTo("2003-01-03T00:00:00.000Z"));
assertThat(termsResponse.field("date").termsFreqs()[0].docFreq(), equalTo(3));
assertThat(termsResponse.field("date").termsFreqs()[1].termAsString(), equalTo("2003-01-01T00:00:00.000Z"));
assertThat(termsResponse.field("date").termsFreqs()[1].docFreq(), equalTo(2));
assertThat(termsResponse.field("date").termsFreqs()[2].termAsString(), equalTo("2003-01-02T00:00:00.000Z"));
assertThat(termsResponse.field("date").termsFreqs()[2].docFreq(), equalTo(1));
}
}

View File

@ -1,6 +0,0 @@
cluster:
routing:
schedule: 100ms
index:
number_of_shards: 5
number_of_replicas: 1

View File

@ -34,8 +34,6 @@ import org.elasticsearch.action.index.IndexResponse
import org.elasticsearch.action.mlt.MoreLikeThisRequest
import org.elasticsearch.action.search.SearchRequest
import org.elasticsearch.action.search.SearchResponse
import org.elasticsearch.action.terms.TermsRequest
import org.elasticsearch.action.terms.TermsResponse
import org.elasticsearch.client.Client
import org.elasticsearch.client.action.count.CountRequestBuilder
import org.elasticsearch.client.action.delete.DeleteRequestBuilder
@ -44,7 +42,6 @@ import org.elasticsearch.client.action.get.GetRequestBuilder
import org.elasticsearch.client.action.index.IndexRequestBuilder
import org.elasticsearch.client.action.search.SearchRequestBuilder
import org.elasticsearch.client.action.support.BaseRequestBuilder
import org.elasticsearch.client.action.terms.TermsRequestBuilder
import org.elasticsearch.client.internal.InternalClient
import org.elasticsearch.common.xcontent.XContentType
import org.elasticsearch.groovy.client.action.GActionFuture
@ -288,28 +285,6 @@ class GClient {
client.search(request, listener)
}
TermsRequestBuilder prepareTerms(String... indices) {
return client.prepareTerms(indices)
}
GActionFuture<TermsResponse> terms(Closure c) {
TermsRequest request = new TermsRequest()
c.resolveStrategy = resolveStrategy
c.setDelegate request
c.call()
terms(request)
}
GActionFuture<TermsResponse> terms(TermsRequest request) {
GActionFuture<TermsResponse> future = new GActionFuture<TermsResponse>(internalClient.threadPool(), request);
client.terms(request, future)
return future
}
void terms(TermsRequest request, ActionListener<TermsResponse> listener) {
client.terms(request, listener)
}
GActionFuture<SearchResponse> moreLikeThis(Closure c) {
MoreLikeThisRequest request = new MoreLikeThisRequest()
c.resolveStrategy = resolveStrategy