mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-09 14:34:43 +00:00
Terms API: Allow to get terms for one or more field. Closes #21.
This commit is contained in:
parent
06cbc0a95b
commit
5d781961a0
2
.idea/dictionaries/kimchy.xml
generated
2
.idea/dictionaries/kimchy.xml
generated
@ -19,9 +19,11 @@
|
||||
<w>flushable</w>
|
||||
<w>formatter</w>
|
||||
<w>formatters</w>
|
||||
<w>freqs</w>
|
||||
<w>indices</w>
|
||||
<w>inet</w>
|
||||
<w>infos</w>
|
||||
<w>iter</w>
|
||||
<w>jgroups</w>
|
||||
<w>joda</w>
|
||||
<w>jsonp</w>
|
||||
|
@ -47,6 +47,7 @@ import org.elasticsearch.action.index.TransportIndexAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchScrollAction;
|
||||
import org.elasticsearch.action.search.type.*;
|
||||
import org.elasticsearch.action.terms.TransportTermsAction;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
@ -80,12 +81,13 @@ public class TransportActionModule extends AbstractModule {
|
||||
bind(TransportIndexAction.class).asEagerSingleton();
|
||||
bind(TransportGetAction.class).asEagerSingleton();
|
||||
bind(TransportDeleteAction.class).asEagerSingleton();
|
||||
bind(TransportCountAction.class).asEagerSingleton();
|
||||
bind(TransportTermsAction.class).asEagerSingleton();
|
||||
|
||||
bind(TransportShardDeleteByQueryAction.class).asEagerSingleton();
|
||||
bind(TransportIndexDeleteByQueryAction.class).asEagerSingleton();
|
||||
bind(TransportDeleteByQueryAction.class).asEagerSingleton();
|
||||
|
||||
bind(TransportCountAction.class).asEagerSingleton();
|
||||
|
||||
bind(TransportSearchCache.class).asEagerSingleton();
|
||||
bind(TransportSearchDfsQueryThenFetchAction.class).asEagerSingleton();
|
||||
|
@ -38,6 +38,8 @@ public class TransportActions {
|
||||
|
||||
public static final String SEARCH_SCROLL = "indices/searchScroll";
|
||||
|
||||
public static final String TERMS = "indices/terms";
|
||||
|
||||
public static class Admin {
|
||||
|
||||
public static class Indices {
|
||||
|
@ -37,19 +37,19 @@ public class IndexStatus implements Iterable<IndexShardStatus> {
|
||||
public static class Docs {
|
||||
public static final Docs UNKNOWN = new Docs();
|
||||
|
||||
int numDocs = -1;
|
||||
int maxDoc = -1;
|
||||
int deletedDocs = -1;
|
||||
long numDocs = -1;
|
||||
long maxDoc = -1;
|
||||
long deletedDocs = -1;
|
||||
|
||||
public int numDocs() {
|
||||
public long numDocs() {
|
||||
return numDocs;
|
||||
}
|
||||
|
||||
public int maxDoc() {
|
||||
public long maxDoc() {
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
public int deletedDocs() {
|
||||
public long deletedDocs() {
|
||||
return deletedDocs;
|
||||
}
|
||||
}
|
||||
@ -83,6 +83,10 @@ public class IndexStatus implements Iterable<IndexShardStatus> {
|
||||
return this.index;
|
||||
}
|
||||
|
||||
/**
|
||||
* A shard id to index shard status map (note, index shard status is the replication shard group that maps
|
||||
* to the shard id).
|
||||
*/
|
||||
public Map<Integer, IndexShardStatus> shards() {
|
||||
return this.indexShards;
|
||||
}
|
||||
|
@ -290,9 +290,9 @@ public class SearchRequest implements ActionRequest {
|
||||
} else {
|
||||
out.writeInt(queryBoost.size());
|
||||
for (TObjectFloatIterator<String> it = queryBoost.iterator(); it.hasNext();) {
|
||||
it.advance();
|
||||
out.writeUTF(it.key());
|
||||
out.writeFloat(it.value());
|
||||
it.advance();
|
||||
}
|
||||
}
|
||||
out.writeInt(types.length);
|
||||
|
@ -254,11 +254,19 @@ public abstract class TransportBroadcastOperationAction<Request extends Broadcas
|
||||
if (request.listenerThreaded() && !alreadyThreaded) {
|
||||
threadPool.execute(new Runnable() {
|
||||
@Override public void run() {
|
||||
listener.onResponse(newResponse(request, shardsResponses, clusterState));
|
||||
try {
|
||||
listener.onResponse(newResponse(request, shardsResponses, clusterState));
|
||||
} catch (Exception e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
} else {
|
||||
listener.onResponse(newResponse(request, shardsResponses, clusterState));
|
||||
try {
|
||||
listener.onResponse(newResponse(request, shardsResponses, clusterState));
|
||||
} catch (Exception e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,100 @@
|
||||
/*
|
||||
* 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 com.google.common.collect.Iterators;
|
||||
import org.elasticsearch.util.io.Streamable;
|
||||
import org.elasticsearch.util.trove.ExtTObjectIntHasMap;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
|
||||
import static org.elasticsearch.action.terms.TermFreq.*;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
*/
|
||||
public class FieldTermsFreq implements Streamable, Iterable<TermFreq> {
|
||||
|
||||
private String fieldName;
|
||||
|
||||
private TermFreq[] termsFreqs;
|
||||
|
||||
private transient ExtTObjectIntHasMap<String> termsFreqMap;
|
||||
|
||||
private FieldTermsFreq() {
|
||||
|
||||
}
|
||||
|
||||
public FieldTermsFreq(String fieldName, TermFreq[] termsFreqs) {
|
||||
this.fieldName = fieldName;
|
||||
this.termsFreqs = termsFreqs;
|
||||
}
|
||||
|
||||
public String fieldName() {
|
||||
return this.fieldName;
|
||||
}
|
||||
|
||||
public TermFreq[] termsFreqs() {
|
||||
return this.termsFreqs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the document frequency of a term, <tt>-1</tt> if the term does not exists.
|
||||
*/
|
||||
public int docFreq(String term) {
|
||||
if (termsFreqMap == null) {
|
||||
ExtTObjectIntHasMap<String> termsFreqMap = new ExtTObjectIntHasMap<String>().defaultReturnValue(-1);
|
||||
for (TermFreq termFreq : termsFreqs) {
|
||||
termsFreqMap.put(termFreq.term(), termFreq.docFreq());
|
||||
}
|
||||
this.termsFreqMap = termsFreqMap;
|
||||
}
|
||||
return termsFreqMap.get(term);
|
||||
}
|
||||
|
||||
@Override public Iterator<TermFreq> iterator() {
|
||||
return Iterators.forArray(termsFreqs);
|
||||
}
|
||||
|
||||
public static FieldTermsFreq readFieldTermsFreq(DataInput in) throws IOException, ClassNotFoundException {
|
||||
FieldTermsFreq fieldTermsFreq = new FieldTermsFreq();
|
||||
fieldTermsFreq.readFrom(in);
|
||||
return fieldTermsFreq;
|
||||
}
|
||||
|
||||
@Override public void readFrom(DataInput in) throws IOException, ClassNotFoundException {
|
||||
fieldName = in.readUTF();
|
||||
termsFreqs = new TermFreq[in.readInt()];
|
||||
for (int i = 0; i < termsFreqs.length; i++) {
|
||||
termsFreqs[i] = readTermFreq(in);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public void writeTo(DataOutput out) throws IOException {
|
||||
out.writeUTF(fieldName);
|
||||
out.writeInt(termsFreqs.length);
|
||||
for (TermFreq termFreq : termsFreqs) {
|
||||
termFreq.writeTo(out);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,180 @@
|
||||
/*
|
||||
* 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 java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
*/
|
||||
public class ShardTermsRequest extends BroadcastShardOperationRequest {
|
||||
|
||||
private String[] fields;
|
||||
|
||||
private String from;
|
||||
|
||||
private String to;
|
||||
|
||||
private boolean fromInclusive = true;
|
||||
|
||||
private boolean toInclusive = false;
|
||||
|
||||
private String prefix;
|
||||
|
||||
private String regexp;
|
||||
|
||||
private int size = 10;
|
||||
|
||||
private boolean convert = true;
|
||||
|
||||
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.convert = request.convert();
|
||||
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 boolean convert() {
|
||||
return convert;
|
||||
}
|
||||
|
||||
public TermsRequest.SortType sortType() {
|
||||
return sortType;
|
||||
}
|
||||
|
||||
public boolean exact() {
|
||||
return this.exact;
|
||||
}
|
||||
|
||||
@Override public void readFrom(DataInput in) throws IOException, ClassNotFoundException {
|
||||
super.readFrom(in);
|
||||
fields = new String[in.readInt()];
|
||||
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.readInt();
|
||||
convert = in.readBoolean();
|
||||
sortType = TermsRequest.SortType.fromValue(in.readByte());
|
||||
exact = in.readBoolean();
|
||||
}
|
||||
|
||||
@Override public void writeTo(DataOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeInt(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.writeInt(size);
|
||||
out.writeBoolean(convert);
|
||||
out.writeByte(sortType.value());
|
||||
out.writeBoolean(exact);
|
||||
}
|
||||
}
|
@ -0,0 +1,110 @@
|
||||
/*
|
||||
* 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.util.gnu.trove.TObjectIntHashMap;
|
||||
import org.elasticsearch.util.gnu.trove.TObjectIntIterator;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
*/
|
||||
public class ShardTermsResponse extends BroadcastShardOperationResponse {
|
||||
|
||||
private Map<String, TObjectIntHashMap<String>> fieldsTermsFreqs = new HashMap<String, TObjectIntHashMap<String>>();
|
||||
|
||||
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<String> termsFreqs) {
|
||||
fieldsTermsFreqs.put(fieldName, termsFreqs);
|
||||
}
|
||||
|
||||
Map<String, TObjectIntHashMap<String>> fieldsTermsFreqs() {
|
||||
return fieldsTermsFreqs;
|
||||
}
|
||||
|
||||
@Override public void readFrom(DataInput in) throws IOException, ClassNotFoundException {
|
||||
super.readFrom(in);
|
||||
numDocs = in.readInt();
|
||||
maxDoc = in.readInt();
|
||||
numDeletedDocs = in.readInt();
|
||||
int size = in.readInt();
|
||||
for (int i = 0; i < size; i++) {
|
||||
String fieldName = in.readUTF();
|
||||
|
||||
TObjectIntHashMap<String> termsFreq = new TObjectIntHashMap<String>();
|
||||
int size1 = in.readInt();
|
||||
for (int j = 0; j < size1; j++) {
|
||||
termsFreq.put(in.readUTF(), in.readInt());
|
||||
}
|
||||
|
||||
fieldsTermsFreqs.put(fieldName, termsFreq);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public void writeTo(final DataOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeInt(numDocs);
|
||||
out.writeInt(maxDoc);
|
||||
out.writeInt(numDeletedDocs);
|
||||
out.writeInt(fieldsTermsFreqs.size());
|
||||
for (Map.Entry<String, TObjectIntHashMap<String>> entry : fieldsTermsFreqs.entrySet()) {
|
||||
out.writeUTF(entry.getKey());
|
||||
out.writeInt(entry.getValue().size());
|
||||
for (TObjectIntIterator<String> it = entry.getValue().iterator(); it.hasNext();) {
|
||||
out.writeUTF(it.key());
|
||||
out.writeInt(it.value());
|
||||
it.advance();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,118 @@
|
||||
/*
|
||||
* 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.util.io.Streamable;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
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 = 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 = 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 String term;
|
||||
|
||||
private int docFreq;
|
||||
|
||||
private TermFreq() {
|
||||
|
||||
}
|
||||
|
||||
TermFreq(String term, int docFreq) {
|
||||
this.term = term;
|
||||
this.docFreq = docFreq;
|
||||
}
|
||||
|
||||
/**
|
||||
* The term.
|
||||
*/
|
||||
public String term() {
|
||||
return term;
|
||||
}
|
||||
|
||||
/**
|
||||
* The document frequency of the term (in how many documents this term exists).
|
||||
*/
|
||||
public int docFreq() {
|
||||
return docFreq;
|
||||
}
|
||||
|
||||
public static TermFreq readTermFreq(DataInput in) throws IOException, ClassNotFoundException {
|
||||
TermFreq termFreq = new TermFreq();
|
||||
termFreq.readFrom(in);
|
||||
return termFreq;
|
||||
}
|
||||
|
||||
@Override public void readFrom(DataInput in) throws IOException, ClassNotFoundException {
|
||||
term = in.readUTF();
|
||||
docFreq = in.readInt();
|
||||
}
|
||||
|
||||
@Override public void writeTo(DataOutput out) throws IOException {
|
||||
out.writeUTF(term);
|
||||
out.writeInt(docFreq);
|
||||
}
|
||||
}
|
@ -0,0 +1,430 @@
|
||||
/*
|
||||
* 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.util.Required;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.elasticsearch.action.Actions.*;
|
||||
|
||||
/**
|
||||
* 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>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 String[] fields;
|
||||
|
||||
private String from;
|
||||
|
||||
private boolean fromInclusive = true;
|
||||
|
||||
private String to;
|
||||
|
||||
private boolean toInclusive = false;
|
||||
|
||||
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. Note, the {@link #fields(String...)} is required.
|
||||
*/
|
||||
public TermsRequest(String... indices) {
|
||||
super(indices, null);
|
||||
}
|
||||
|
||||
@Override public ActionRequestValidationException validate() {
|
||||
ActionRequestValidationException validationException = super.validate();
|
||||
if (fields == null || fields.length == 0) {
|
||||
validationException = addValidationError("fields is missing", validationException);
|
||||
}
|
||||
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.
|
||||
*/
|
||||
@Required public TermsRequest fields(String... fields) {
|
||||
this.fields = fields;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* The lower bound (lex) term from which the iteration will start. Defaults to start from the
|
||||
* first.
|
||||
*/
|
||||
public String from() {
|
||||
return from;
|
||||
}
|
||||
|
||||
/**
|
||||
* The lower bound (lex) term from which the iteration will start. Defaults to start from the
|
||||
* first.
|
||||
*/
|
||||
public TermsRequest from(String from) {
|
||||
this.from = from;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should the first from (if set using {@link #from(String)} 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(String)} 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 (lex) term to which the iteration will end. Defaults to unbound (<tt>null</tt>).
|
||||
*/
|
||||
public String to() {
|
||||
return to;
|
||||
}
|
||||
|
||||
/**
|
||||
* The upper bound (lex) term to which the iteration will end. Defaults to unbound (<tt>null</tt>).
|
||||
*/
|
||||
public TermsRequest to(String to) {
|
||||
this.to = to;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should the last to (if set using {@link #to(String)} be inclusive or not. Defaults to
|
||||
* <tt>true</tt>.
|
||||
*/
|
||||
public boolean toInclusive() {
|
||||
return toInclusive;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should the last to (if set using {@link #to(String)} 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 void regexp(String regexp) {
|
||||
this.regexp = regexp;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should an attempt be made to convert the {@link #to(String)} and {@link #from(String)}.
|
||||
* Defaults to <tt>true</tt>.
|
||||
*/
|
||||
public boolean convert() {
|
||||
return convert;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should an attempt be made to convert the {@link #to(String)} and {@link #from(String)}.
|
||||
* Defaults to <tt>true</tt>.
|
||||
*/
|
||||
public TermsRequest convert(boolean convert) {
|
||||
this.convert = convert;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* The type of sorting for term / doc freq. Can either sort on term (lex) or doc frequncy. 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 frequncy. Defaults to
|
||||
* {@link TermsRequest.SortType#TERM}.
|
||||
*/
|
||||
public TermsRequest sortType(SortType sortType) {
|
||||
this.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 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(DataOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeInt(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.writeInt(size);
|
||||
out.writeBoolean(convert);
|
||||
out.writeByte(sortType.value());
|
||||
out.writeInt(minFreq);
|
||||
out.writeInt(maxFreq);
|
||||
out.writeBoolean(exact);
|
||||
}
|
||||
|
||||
@Override public void readFrom(DataInput in) throws IOException, ClassNotFoundException {
|
||||
super.readFrom(in);
|
||||
fields = new String[in.readInt()];
|
||||
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.readInt();
|
||||
convert = in.readBoolean();
|
||||
sortType = TermsRequest.SortType.fromValue(in.readByte());
|
||||
minFreq = in.readInt();
|
||||
maxFreq = in.readInt();
|
||||
exact = in.readBoolean();
|
||||
}
|
||||
}
|
@ -0,0 +1,146 @@
|
||||
/*
|
||||
* 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 com.google.common.collect.Iterators;
|
||||
import org.elasticsearch.action.support.broadcast.BroadcastOperationResponse;
|
||||
|
||||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
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 successfulShards, int failedShards, FieldTermsFreq[] fieldsTermsFreq,
|
||||
long numDocs, long maxDoc, long numDeletedDocs) {
|
||||
super(successfulShards, failedShards);
|
||||
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 maximum number of documents (including deletions).
|
||||
*/
|
||||
public long maxDoc() {
|
||||
return this.maxDoc;
|
||||
}
|
||||
|
||||
/**
|
||||
* The number of deleted docs.
|
||||
*/
|
||||
public long deletedDocs() {
|
||||
return this.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;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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(DataInput in) throws IOException, ClassNotFoundException {
|
||||
super.readFrom(in);
|
||||
numDocs = in.readLong();
|
||||
maxDoc = in.readLong();
|
||||
numDeletedDocs = in.readLong();
|
||||
fieldsTermsFreq = new FieldTermsFreq[in.readInt()];
|
||||
for (int i = 0; i < fieldsTermsFreq.length; i++) {
|
||||
fieldsTermsFreq[i] = readFieldTermsFreq(in);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public void writeTo(DataOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeLong(numDocs);
|
||||
out.writeLong(maxDoc);
|
||||
out.writeLong(numDeletedDocs);
|
||||
out.writeInt(fieldsTermsFreq.length);
|
||||
for (FieldTermsFreq fieldTermsFreq : fieldsTermsFreq) {
|
||||
fieldTermsFreq.writeTo(out);
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,332 @@
|
||||
/*
|
||||
* 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 com.google.inject.Inject;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermDocs;
|
||||
import org.apache.lucene.index.TermEnum;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.action.TransportActions;
|
||||
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.index.IndexService;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.util.BoundedTreeSet;
|
||||
import org.elasticsearch.util.gnu.trove.TObjectIntHashMap;
|
||||
import org.elasticsearch.util.gnu.trove.TObjectIntIterator;
|
||||
import org.elasticsearch.util.settings.Settings;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.concurrent.atomic.AtomicReferenceArray;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.elasticsearch.action.Actions.*;
|
||||
|
||||
/**
|
||||
* @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;
|
||||
ShardTermsResponse aggregator = null;
|
||||
for (int i = 0; i < shardsResponses.length(); i++) {
|
||||
ShardTermsResponse shardResponse = (ShardTermsResponse) shardsResponses.get(i);
|
||||
if (shardResponse == null) {
|
||||
failedShards++;
|
||||
} else {
|
||||
if (aggregator == null) {
|
||||
aggregator = shardResponse;
|
||||
} else {
|
||||
for (Map.Entry<String, TObjectIntHashMap<String>> entry : shardResponse.fieldsTermsFreqs().entrySet()) {
|
||||
String fieldName = entry.getKey();
|
||||
TObjectIntHashMap<String> termsFreqs = aggregator.fieldsTermsFreqs().get(fieldName);
|
||||
if (termsFreqs == null) {
|
||||
termsFreqs = new TObjectIntHashMap<String>();
|
||||
aggregator.put(fieldName, termsFreqs);
|
||||
}
|
||||
for (TObjectIntIterator<String> it = entry.getValue().iterator(); it.hasNext();) {
|
||||
it.advance();
|
||||
termsFreqs.adjustOrPutValue(it.key(), it.value(), it.value());
|
||||
}
|
||||
}
|
||||
}
|
||||
numDocs += shardResponse.numDocs();
|
||||
maxDoc += shardResponse.maxDoc();
|
||||
numDeletedDocs += shardResponse.numDeletedDocs();
|
||||
successfulShards++;
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, NavigableSet<TermFreq>> fieldTermsFreqs = new HashMap<String, NavigableSet<TermFreq>>();
|
||||
if (aggregator != null) {
|
||||
for (Map.Entry<String, TObjectIntHashMap<String>> entry : aggregator.fieldsTermsFreqs().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<String> 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(successfulShards, failedShards, 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()) {
|
||||
TObjectIntHashMap<String> termsFreqs = new TObjectIntHashMap<String>();
|
||||
|
||||
FieldMapper fieldMapper = indexService.mapperService().smartNameFieldMapper(fieldName);
|
||||
String indexFieldName = fieldName;
|
||||
if (fieldMapper != null) {
|
||||
indexFieldName = fieldMapper.indexName();
|
||||
}
|
||||
indexFieldName = StringHelper.intern(indexFieldName);
|
||||
|
||||
// setup the to and from
|
||||
String from = request.from();
|
||||
if (from == null) {
|
||||
from = request.prefix();
|
||||
} else {
|
||||
if (request.convert()) {
|
||||
if (fieldMapper != null) {
|
||||
from = fieldMapper.indexedValue(from);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (from == null) {
|
||||
from = "";
|
||||
}
|
||||
Term fromTerm = new Term(indexFieldName, from);
|
||||
|
||||
String to = request.to();
|
||||
if (to != null && request.convert() && 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) {
|
||||
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;
|
||||
}
|
||||
// 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++;
|
||||
}
|
||||
} else if (request.sortType() == TermsRequest.SortType.FREQ) {
|
||||
BoundedTreeSet<TermFreq> sortedFreq = new BoundedTreeSet<TermFreq>(TermFreq.freqComparator(), request.size());
|
||||
while (true) {
|
||||
Term term = termEnum.term();
|
||||
// have we reached the end?
|
||||
if (term == null || indexFieldName != term.field()) { // StirngHelper.intern
|
||||
break;
|
||||
}
|
||||
// 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++;
|
||||
}
|
||||
}
|
||||
}
|
||||
sortedFreq.add(new TermFreq(term.text(), docFreq));
|
||||
if (!termEnum.next()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
for (TermFreq termFreq : sortedFreq) {
|
||||
termsFreqs.put(termFreq.term(), termFreq.docFreq());
|
||||
}
|
||||
}
|
||||
|
||||
response.put(fieldName, termsFreqs);
|
||||
} catch (Exception e) {
|
||||
logger.debug("Failed to get term enum from term [" + fromTerm + "]", e);
|
||||
} finally {
|
||||
if (termEnum != null) {
|
||||
try {
|
||||
termEnum.close();
|
||||
} catch (IOException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return response;
|
||||
} finally {
|
||||
if (termDocs != null) {
|
||||
try {
|
||||
termDocs.close();
|
||||
} catch (IOException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
searcher.release();
|
||||
}
|
||||
}
|
||||
|
||||
@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 boolean accumulateExceptions() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override protected GroupShardsIterator shards(TermsRequest request, ClusterState clusterState) {
|
||||
return indicesService.searchShards(clusterState, processIndices(clusterState, request.indices()), request.queryHint());
|
||||
}
|
||||
}
|
@ -34,6 +34,8 @@ import org.elasticsearch.action.index.IndexResponse;
|
||||
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;
|
||||
|
||||
/**
|
||||
* A client provides a one stop interface for performing actions/operations against the cluster.
|
||||
@ -263,4 +265,10 @@ public interface Client {
|
||||
* @see Requests#searchScrollRequest(String)
|
||||
*/
|
||||
void execSearchScroll(SearchScrollRequest request, ActionListener<SearchResponse> listener);
|
||||
|
||||
ActionFuture<TermsResponse> terms(TermsRequest request);
|
||||
|
||||
ActionFuture<TermsResponse> terms(TermsRequest request, ActionListener<TermsResponse> listener);
|
||||
|
||||
void execTerms(TermsRequest request, ActionListener<TermsResponse> listener);
|
||||
}
|
@ -39,6 +39,7 @@ import org.elasticsearch.action.get.GetRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchScrollRequest;
|
||||
import org.elasticsearch.action.terms.TermsRequest;
|
||||
|
||||
/**
|
||||
* A handy one stop shop for creating requests (make sure to import static this class).
|
||||
@ -107,6 +108,10 @@ public class Requests {
|
||||
return new CountRequest(indices);
|
||||
}
|
||||
|
||||
public static TermsRequest termsRequest(String... indices) {
|
||||
return new TermsRequest(indices);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a search request against one or more indices. Note, the search source must be set either using the
|
||||
* actual JSON search source, or the {@link org.elasticsearch.search.builder.SearchSourceBuilder}.
|
||||
@ -201,7 +206,7 @@ public class Requests {
|
||||
*
|
||||
* @param indices The indices to optimize. Use <tt>null</tt> or <tt>_all</tt> to execute against all indices
|
||||
* @return The optimize request
|
||||
* @see org.elasticsearch.client.IndicesAdminClient#optimize(org.elasticsearch.action.admin.indices.flush.FlushRequest)
|
||||
* @see org.elasticsearch.client.IndicesAdminClient#optimize(org.elasticsearch.action.admin.indices.optimize.OptimizeRequest)
|
||||
*/
|
||||
public static OptimizeRequest optimizeRequest(String... indices) {
|
||||
return new OptimizeRequest(indices);
|
||||
|
@ -38,6 +38,9 @@ import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.index.TransportIndexAction;
|
||||
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.Client;
|
||||
import org.elasticsearch.util.component.AbstractComponent;
|
||||
@ -64,10 +67,13 @@ public class ServerClient extends AbstractComponent implements Client {
|
||||
|
||||
private final TransportSearchScrollAction searchScrollAction;
|
||||
|
||||
private final TransportTermsAction termsAction;
|
||||
|
||||
@Inject public ServerClient(Settings settings, ServerAdminClient admin,
|
||||
TransportIndexAction indexAction, TransportDeleteAction deleteAction,
|
||||
TransportDeleteByQueryAction deleteByQueryAction, TransportGetAction getAction, TransportCountAction countAction,
|
||||
TransportSearchAction searchAction, TransportSearchScrollAction searchScrollAction) {
|
||||
TransportSearchAction searchAction, TransportSearchScrollAction searchScrollAction,
|
||||
TransportTermsAction termsAction) {
|
||||
super(settings);
|
||||
this.admin = admin;
|
||||
this.indexAction = indexAction;
|
||||
@ -77,6 +83,7 @@ public class ServerClient extends AbstractComponent implements Client {
|
||||
this.countAction = countAction;
|
||||
this.searchAction = searchAction;
|
||||
this.searchScrollAction = searchScrollAction;
|
||||
this.termsAction = termsAction;
|
||||
}
|
||||
|
||||
@Override public void close() {
|
||||
@ -170,4 +177,16 @@ public class ServerClient extends AbstractComponent implements Client {
|
||||
@Override public void execSearchScroll(SearchScrollRequest request, ActionListener<SearchResponse> listener) {
|
||||
searchScrollAction.execute(request, listener);
|
||||
}
|
||||
|
||||
@Override public ActionFuture<TermsResponse> terms(TermsRequest request) {
|
||||
return termsAction.submit(request);
|
||||
}
|
||||
|
||||
@Override public ActionFuture<TermsResponse> terms(TermsRequest request, ActionListener<TermsResponse> listener) {
|
||||
return termsAction.submit(request, listener);
|
||||
}
|
||||
|
||||
@Override public void execTerms(TermsRequest request, ActionListener<TermsResponse> listener) {
|
||||
termsAction.execute(request, listener);
|
||||
}
|
||||
}
|
||||
|
@ -39,6 +39,8 @@ import org.elasticsearch.action.index.IndexResponse;
|
||||
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.Client;
|
||||
import org.elasticsearch.client.transport.action.ClientTransportActionModule;
|
||||
@ -283,4 +285,16 @@ public class TransportClient implements Client {
|
||||
@Override public void execSearchScroll(SearchScrollRequest request, ActionListener<SearchResponse> listener) {
|
||||
internalClient.execSearchScroll(request, listener);
|
||||
}
|
||||
|
||||
@Override public ActionFuture<TermsResponse> terms(TermsRequest request) {
|
||||
return internalClient.terms(request);
|
||||
}
|
||||
|
||||
@Override public ActionFuture<TermsResponse> terms(TermsRequest request, ActionListener<TermsResponse> listener) {
|
||||
return internalClient.terms(request, listener);
|
||||
}
|
||||
|
||||
@Override public void execTerms(TermsRequest request, ActionListener<TermsResponse> listener) {
|
||||
internalClient.terms(request, listener);
|
||||
}
|
||||
}
|
||||
|
@ -40,6 +40,7 @@ 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;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
@ -54,6 +55,7 @@ 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();
|
||||
|
@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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 com.google.inject.Inject;
|
||||
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.transport.TransportService;
|
||||
import org.elasticsearch.util.settings.Settings;
|
||||
|
||||
/**
|
||||
* @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;
|
||||
}
|
||||
}
|
@ -36,6 +36,8 @@ import org.elasticsearch.action.index.IndexResponse;
|
||||
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.Client;
|
||||
import org.elasticsearch.client.transport.TransportClientNodesService;
|
||||
@ -46,6 +48,7 @@ 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.cluster.node.Node;
|
||||
import org.elasticsearch.util.component.AbstractComponent;
|
||||
import org.elasticsearch.util.settings.Settings;
|
||||
@ -73,10 +76,13 @@ public class InternalTransportClient extends AbstractComponent implements Client
|
||||
|
||||
private final ClientTransportSearchScrollAction searchScrollAction;
|
||||
|
||||
private final ClientTransportTermsAction termsAction;
|
||||
|
||||
@Inject public InternalTransportClient(Settings settings, TransportClientNodesService nodesService, InternalTransportAdminClient adminClient,
|
||||
ClientTransportIndexAction indexAction, ClientTransportDeleteAction deleteAction, ClientTransportGetAction getAction,
|
||||
ClientTransportDeleteByQueryAction deleteByQueryAction, ClientTransportCountAction countAction,
|
||||
ClientTransportSearchAction searchAction, ClientTransportSearchScrollAction searchScrollAction) {
|
||||
ClientTransportSearchAction searchAction, ClientTransportSearchScrollAction searchScrollAction,
|
||||
ClientTransportTermsAction termsAction) {
|
||||
super(settings);
|
||||
this.nodesService = nodesService;
|
||||
this.adminClient = adminClient;
|
||||
@ -88,6 +94,7 @@ public class InternalTransportClient extends AbstractComponent implements Client
|
||||
this.countAction = countAction;
|
||||
this.searchAction = searchAction;
|
||||
this.searchScrollAction = searchScrollAction;
|
||||
this.termsAction = termsAction;
|
||||
}
|
||||
|
||||
@Override public void close() {
|
||||
@ -272,4 +279,29 @@ public class InternalTransportClient extends AbstractComponent implements Client
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override public ActionFuture<TermsResponse> terms(final TermsRequest request) {
|
||||
return nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<TermsResponse>>() {
|
||||
@Override public ActionFuture<TermsResponse> doWithNode(Node node) throws ElasticSearchException {
|
||||
return termsAction.submit(node, request);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override public ActionFuture<TermsResponse> terms(final TermsRequest request, final ActionListener<TermsResponse> listener) {
|
||||
return nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<TermsResponse>>() {
|
||||
@Override public ActionFuture<TermsResponse> doWithNode(Node node) throws ElasticSearchException {
|
||||
return termsAction.submit(node, request, listener);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override public void execTerms(final TermsRequest request, final ActionListener<TermsResponse> listener) {
|
||||
nodesService.execute(new TransportClientNodesService.NodeCallback<ActionFuture<Void>>() {
|
||||
@Override public ActionFuture<Void> doWithNode(Node node) throws ElasticSearchException {
|
||||
termsAction.execute(node, request, listener);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -49,6 +49,12 @@ public interface HttpRequest {
|
||||
|
||||
String param(String key);
|
||||
|
||||
float paramAsFloat(String key, float defaultValue);
|
||||
|
||||
int paramAsInt(String key, int defaultValue);
|
||||
|
||||
boolean paramAsBoolean(String key, boolean defaultValue);
|
||||
|
||||
List<String> params(String key);
|
||||
|
||||
Map<String, List<String>> params();
|
||||
|
@ -40,6 +40,7 @@ import org.elasticsearch.http.action.get.HttpGetAction;
|
||||
import org.elasticsearch.http.action.index.HttpIndexAction;
|
||||
import org.elasticsearch.http.action.main.HttpMainAction;
|
||||
import org.elasticsearch.http.action.search.HttpSearchAction;
|
||||
import org.elasticsearch.http.action.terms.HttpTermsAction;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
@ -77,6 +78,7 @@ public class HttpActionModule extends AbstractModule {
|
||||
bind(HttpDeleteByQueryAction.class).asEagerSingleton();
|
||||
|
||||
bind(HttpCountAction.class).asEagerSingleton();
|
||||
bind(HttpTermsAction.class).asEagerSingleton();
|
||||
|
||||
bind(HttpSearchAction.class).asEagerSingleton();
|
||||
}
|
||||
|
@ -48,7 +48,7 @@ public class HttpNodesInfoAction extends BaseHttpServerHandler {
|
||||
|
||||
@Override public void handleRequest(final HttpRequest request, final HttpChannel channel) {
|
||||
String[] nodesIds = HttpActions.splitNodes(request.param("nodeId"));
|
||||
final boolean includeSettings = HttpActions.paramAsBoolean("settings", false);
|
||||
final boolean includeSettings = request.paramAsBoolean("settings", false);
|
||||
NodesInfoRequest nodesInfoRequest = new NodesInfoRequest(nodesIds);
|
||||
nodesInfoRequest.listenerThreaded(false);
|
||||
client.admin().cluster().execNodesInfo(nodesInfoRequest, new ActionListener<NodesInfoResponse>() {
|
||||
|
@ -56,7 +56,7 @@ public class HttpFlushAction extends BaseHttpServerHandler {
|
||||
operationThreading = BroadcastOperationThreading.THREAD_PER_SHARD;
|
||||
}
|
||||
flushRequest.operationThreading(operationThreading);
|
||||
flushRequest.refresh(HttpActions.paramAsBoolean("refresh", false));
|
||||
flushRequest.refresh(request.paramAsBoolean("refresh", flushRequest.refresh()));
|
||||
client.admin().indices().execFlush(flushRequest, new ActionListener<FlushResponse>() {
|
||||
@Override public void onResponse(FlushResponse response) {
|
||||
try {
|
||||
|
@ -49,11 +49,11 @@ public class HttpOptimizeAction extends BaseHttpServerHandler {
|
||||
@Override public void handleRequest(final HttpRequest request, final HttpChannel channel) {
|
||||
OptimizeRequest optimizeRequest = new OptimizeRequest(HttpActions.splitIndices(request.param("index")));
|
||||
try {
|
||||
optimizeRequest.waitForMerge(HttpActions.paramAsBoolean(request.param("waitForMerge"), true));
|
||||
optimizeRequest.maxNumSegments(HttpActions.paramAsInt(request.param("maxNumSegments"), -1));
|
||||
optimizeRequest.onlyExpungeDeletes(HttpActions.paramAsBoolean(request.param("onlyExpungeDeletes"), false));
|
||||
optimizeRequest.flush(HttpActions.paramAsBoolean(request.param("flush"), false));
|
||||
optimizeRequest.refresh(HttpActions.paramAsBoolean(request.param("refresh"), false));
|
||||
optimizeRequest.waitForMerge(request.paramAsBoolean("waitForMerge", optimizeRequest.waitForMerge()));
|
||||
optimizeRequest.maxNumSegments(request.paramAsInt("maxNumSegments", optimizeRequest.maxNumSegments()));
|
||||
optimizeRequest.onlyExpungeDeletes(request.paramAsBoolean("onlyExpungeDeletes", optimizeRequest.onlyExpungeDeletes()));
|
||||
optimizeRequest.flush(request.paramAsBoolean("flush", optimizeRequest.flush()));
|
||||
optimizeRequest.refresh(request.paramAsBoolean("refresh", optimizeRequest.refresh()));
|
||||
|
||||
// we just send back a response, no need to fork a listener
|
||||
optimizeRequest.listenerThreaded(false);
|
||||
|
@ -84,6 +84,7 @@ public class HttpIndicesStatusAction extends BaseHttpServerHandler {
|
||||
builder.field("estimatedFlushableMemorySize", indexStatus.estimatedFlushableMemorySize().toString());
|
||||
builder.field("estimatedFlushableMemorySizeInBytes", indexStatus.estimatedFlushableMemorySize().bytes());
|
||||
builder.field("translogOperations", indexStatus.translogOperations());
|
||||
|
||||
builder.startObject("docs");
|
||||
builder.field("numDocs", indexStatus.docs().numDocs());
|
||||
builder.field("maxDoc", indexStatus.docs().maxDoc());
|
||||
|
@ -66,7 +66,7 @@ public class HttpCountAction extends BaseHttpServerHandler {
|
||||
countRequest.querySource(HttpActions.parseQuerySource(request));
|
||||
countRequest.queryParserName(request.param("queryParserName"));
|
||||
countRequest.queryHint(request.param("queryHint"));
|
||||
countRequest.minScore(paramAsFloat(request.param("minScore"), DEFAULT_MIN_SCORE));
|
||||
countRequest.minScore(request.paramAsFloat("minScore", DEFAULT_MIN_SCORE));
|
||||
String typesParam = request.param("type");
|
||||
if (typesParam != null) {
|
||||
countRequest.types(splitTypes(typesParam));
|
||||
|
@ -51,7 +51,6 @@ public class HttpSearchAction extends BaseHttpServerHandler {
|
||||
|
||||
public final static Pattern fieldsPattern;
|
||||
|
||||
|
||||
static {
|
||||
fieldsPattern = Pattern.compile(",");
|
||||
}
|
||||
|
@ -84,25 +84,4 @@ public class HttpActions {
|
||||
}
|
||||
return nodesIdsPattern.split(nodes);
|
||||
}
|
||||
|
||||
public static float paramAsFloat(String floatValue, float defaultValue) {
|
||||
if (floatValue == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
return Float.parseFloat(floatValue);
|
||||
}
|
||||
|
||||
public static int paramAsInt(String intValue, int defaultValue) {
|
||||
if (intValue == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
return Integer.parseInt(intValue);
|
||||
}
|
||||
|
||||
public static boolean paramAsBoolean(String booleanValue, boolean defaultValue) {
|
||||
if (booleanValue == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
return Boolean.parseBoolean(booleanValue);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,164 @@
|
||||
/*
|
||||
* 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.http.action.terms;
|
||||
|
||||
import com.google.inject.Inject;
|
||||
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.http.*;
|
||||
import org.elasticsearch.http.action.support.HttpJsonBuilder;
|
||||
import org.elasticsearch.util.json.JsonBuilder;
|
||||
import org.elasticsearch.util.settings.Settings;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.elasticsearch.http.HttpResponse.Status.*;
|
||||
import static org.elasticsearch.http.action.support.HttpActions.*;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
*/
|
||||
public class HttpTermsAction extends BaseHttpServerHandler {
|
||||
|
||||
private final static Pattern fieldsPattern;
|
||||
|
||||
static {
|
||||
fieldsPattern = Pattern.compile(",");
|
||||
}
|
||||
|
||||
@Inject public HttpTermsAction(Settings settings, HttpServer httpService, Client client) {
|
||||
super(settings, client);
|
||||
httpService.registerHandler(HttpRequest.Method.POST, "/_terms", this);
|
||||
httpService.registerHandler(HttpRequest.Method.GET, "/_terms", this);
|
||||
httpService.registerHandler(HttpRequest.Method.POST, "/{index}/_terms", this);
|
||||
httpService.registerHandler(HttpRequest.Method.GET, "/{index}/_terms", this);
|
||||
}
|
||||
|
||||
@Override public void handleRequest(final HttpRequest request, final HttpChannel 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("operationThreading"), 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);
|
||||
|
||||
List<String> fields = request.params("field");
|
||||
if (fields == null) {
|
||||
fields = new ArrayList<String>();
|
||||
}
|
||||
String sField = request.param("fields");
|
||||
if (sField != null) {
|
||||
String[] sFields = fieldsPattern.split(sField);
|
||||
if (sFields != null) {
|
||||
for (String field : sFields) {
|
||||
fields.add(field);
|
||||
}
|
||||
}
|
||||
}
|
||||
termsRequest.fields(fields.toArray(new String[fields.size()]));
|
||||
|
||||
termsRequest.from(request.param("from"));
|
||||
termsRequest.to(request.param("to"));
|
||||
termsRequest.fromInclusive(request.paramAsBoolean("fromInclusive", termsRequest.fromInclusive()));
|
||||
termsRequest.toInclusive(request.paramAsBoolean("toInclusive", termsRequest.toInclusive()));
|
||||
termsRequest.exact(request.paramAsBoolean("exact", termsRequest.exact()));
|
||||
termsRequest.minFreq(request.paramAsInt("minFreq", termsRequest.minFreq()));
|
||||
termsRequest.maxFreq(request.paramAsInt("maxFreq", termsRequest.maxFreq()));
|
||||
termsRequest.size(request.paramAsInt("size", termsRequest.size()));
|
||||
termsRequest.convert(request.paramAsBoolean("convert", termsRequest.convert()));
|
||||
termsRequest.prefix(request.param("prefix"));
|
||||
termsRequest.regexp(request.param("regexp"));
|
||||
termsRequest.sortType(TermsRequest.SortType.fromString(request.param("sort"), termsRequest.sortType()));
|
||||
} catch (Exception e) {
|
||||
try {
|
||||
channel.sendResponse(new JsonHttpResponse(request, BAD_REQUEST, JsonBuilder.jsonBuilder().startObject().field("error", e.getMessage()).endObject()));
|
||||
} catch (IOException e1) {
|
||||
logger.error("Failed to send failure response", e1);
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
||||
client.execTerms(termsRequest, new ActionListener<TermsResponse>() {
|
||||
@Override public void onResponse(TermsResponse response) {
|
||||
try {
|
||||
JsonBuilder builder = HttpJsonBuilder.cached(request);
|
||||
builder.startObject();
|
||||
|
||||
builder.startObject("_shards");
|
||||
builder.field("total", response.totalShards());
|
||||
builder.field("successful", response.successfulShards());
|
||||
builder.field("failed", response.failedShards());
|
||||
builder.endObject();
|
||||
|
||||
builder.startObject("docs");
|
||||
builder.field("numDocs", response.numDocs());
|
||||
builder.field("maxDoc", response.maxDoc());
|
||||
builder.field("deletedDocs", response.deletedDocs());
|
||||
builder.endObject();
|
||||
|
||||
builder.startObject("fields");
|
||||
for (FieldTermsFreq fieldTermsFreq : response.fields()) {
|
||||
builder.startObject(fieldTermsFreq.fieldName());
|
||||
|
||||
builder.startObject("terms");
|
||||
for (TermFreq termFreq : fieldTermsFreq.termsFreqs()) {
|
||||
builder.startObject(termFreq.term());
|
||||
builder.field("docFreq", termFreq.docFreq());
|
||||
builder.endObject();
|
||||
}
|
||||
builder.endObject();
|
||||
|
||||
builder.endObject();
|
||||
}
|
||||
builder.endObject();
|
||||
|
||||
builder.endObject();
|
||||
channel.sendResponse(new JsonHttpResponse(request, OK, builder));
|
||||
} catch (Exception e) {
|
||||
onFailure(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public void onFailure(Throwable e) {
|
||||
try {
|
||||
channel.sendResponse(new JsonThrowableHttpResponse(request, e));
|
||||
} catch (IOException e1) {
|
||||
logger.error("Failed to send failure response", e1);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override public boolean spawn() {
|
||||
return false;
|
||||
}
|
||||
}
|
@ -20,6 +20,7 @@
|
||||
package org.elasticsearch.http.netty;
|
||||
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
import org.elasticsearch.http.HttpRequest;
|
||||
import org.jboss.netty.buffer.ChannelBuffer;
|
||||
import org.jboss.netty.handler.codec.http.HttpHeaders;
|
||||
@ -98,6 +99,42 @@ public class NettyHttpRequest implements HttpRequest {
|
||||
return request.getHeader(HttpHeaders.Names.COOKIE);
|
||||
}
|
||||
|
||||
@Override public float paramAsFloat(String key, float defaultValue) {
|
||||
String sValue = param(key);
|
||||
if (sValue == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
try {
|
||||
return Float.parseFloat(sValue);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new ElasticSearchIllegalArgumentException("Failed to parse float parameter [" + key + "] with value [" + sValue + "]", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public int paramAsInt(String key, int defaultValue) {
|
||||
String sValue = param(key);
|
||||
if (sValue == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
try {
|
||||
return Integer.parseInt(sValue);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new ElasticSearchIllegalArgumentException("Failed to parse int parameter [" + key + "] with value [" + sValue + "]", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public boolean paramAsBoolean(String key, boolean defaultValue) {
|
||||
String sValue = param(key);
|
||||
if (sValue == null) {
|
||||
return defaultValue;
|
||||
}
|
||||
try {
|
||||
return Boolean.valueOf(sValue);
|
||||
} catch (NumberFormatException e) {
|
||||
throw new ElasticSearchIllegalArgumentException("Failed to parse boolean parameter [" + key + "] with value [" + sValue + "]", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public String param(String key) {
|
||||
List<String> keyParams = params(key);
|
||||
if (keyParams == null || keyParams.isEmpty()) {
|
||||
|
@ -314,6 +314,9 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine,
|
||||
} else {
|
||||
indexWriter.optimize(maxNumberOfSegments, optimize.waitForMerge());
|
||||
}
|
||||
// once we did the optimization, we are "dirty" since we removed deletes potentially which
|
||||
// affects TermEnum
|
||||
dirty = true;
|
||||
} catch (Exception e) {
|
||||
throw new OptimizeFailedEngineException(shardId, e);
|
||||
} finally {
|
||||
|
@ -60,6 +60,10 @@ public class FieldMappers implements Iterable<FieldMapper> {
|
||||
return fieldMappers.get(0);
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return fieldMappers.isEmpty();
|
||||
}
|
||||
|
||||
public ImmutableList<FieldMapper> mappers() {
|
||||
return this.fieldMappers;
|
||||
}
|
||||
|
@ -228,6 +228,18 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
||||
return fullNameFieldMappers.get(fullName);
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #smartNameFieldMappers(String)} but returns the first field mapper for it. Returns
|
||||
* <tt>null</tt> if there is none.
|
||||
*/
|
||||
public FieldMapper smartNameFieldMapper(String smartName) {
|
||||
FieldMappers fieldMappers = smartNameFieldMappers(smartName);
|
||||
if (fieldMappers != null) {
|
||||
return fieldMappers.mapper();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Same as {@link #smartName(String)}, except it returns just the field mappers.
|
||||
*/
|
||||
@ -317,14 +329,39 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
||||
this.docMapper = docMapper;
|
||||
}
|
||||
|
||||
/**
|
||||
* Has at least one mapper for the field.
|
||||
*/
|
||||
public boolean hasMapper() {
|
||||
return !fieldMappers.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* The first mapper for the smart named field.
|
||||
*/
|
||||
public FieldMapper mapper() {
|
||||
return fieldMappers.mapper();
|
||||
}
|
||||
|
||||
/**
|
||||
* All the field mappers for the smart name field.
|
||||
*/
|
||||
public FieldMappers fieldMappers() {
|
||||
return fieldMappers;
|
||||
}
|
||||
|
||||
/**
|
||||
* If the smart name was a typed field, with a type that we resolved, will return
|
||||
* <tt>true</tt>.
|
||||
*/
|
||||
public boolean hasDocMapper() {
|
||||
return docMapper != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* If the smart name was a typed field, with a type that we resolved, will return
|
||||
* the document mapper for it.
|
||||
*/
|
||||
public DocumentMapper docMapper() {
|
||||
return docMapper;
|
||||
}
|
||||
|
@ -87,7 +87,7 @@ public abstract class JsonNumberFieldMapper<T extends Number> extends JsonFieldM
|
||||
}
|
||||
|
||||
/**
|
||||
* Override the defualt behavior (to return the string, and reutrn the actual Number instance).
|
||||
* Override the default behavior (to return the string, and return the actual Number instance).
|
||||
*/
|
||||
@Override public Object valueForSearch(Fieldable field) {
|
||||
return value(field);
|
||||
|
@ -27,7 +27,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -71,10 +70,9 @@ public class PrefixJsonFilterParser extends AbstractIndexComponent implements Js
|
||||
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
fieldName = fieldMapper.indexName();
|
||||
value = fieldMapper.indexedValue(value);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
fieldName = smartNameFieldMappers.mapper().indexName();
|
||||
value = smartNameFieldMappers.mapper().indexedValue(value);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -28,7 +28,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -91,10 +90,9 @@ public class PrefixJsonQueryParser extends AbstractIndexComponent implements Jso
|
||||
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
fieldName = fieldMapper.indexName();
|
||||
value = fieldMapper.indexedValue(value);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
fieldName = smartNameFieldMappers.mapper().indexName();
|
||||
value = smartNameFieldMappers.mapper().indexedValue(value);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -94,9 +93,8 @@ public class RangeJsonFilterParser extends AbstractIndexComponent implements Jso
|
||||
Filter filter = null;
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
filter = fieldMapper.rangeFilter(from, to, includeLower, includeUpper);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
filter = smartNameFieldMappers.mapper().rangeFilter(from, to, includeLower, includeUpper);
|
||||
}
|
||||
}
|
||||
if (filter == null) {
|
||||
|
@ -26,7 +26,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -97,9 +96,8 @@ public class RangeJsonQueryParser extends AbstractIndexComponent implements Json
|
||||
Query query = null;
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
query = fieldMapper.rangeQuery(from, to, includeLower, includeUpper);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
query = smartNameFieldMappers.mapper().rangeQuery(from, to, includeLower, includeUpper);
|
||||
}
|
||||
}
|
||||
if (query == null) {
|
||||
|
@ -27,7 +27,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -91,10 +90,9 @@ public class SpanTermJsonQueryParser extends AbstractIndexComponent implements J
|
||||
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
fieldName = fieldMapper.indexName();
|
||||
value = fieldMapper.indexedValue(value);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
fieldName = smartNameFieldMappers.mapper().indexName();
|
||||
value = smartNameFieldMappers.mapper().indexedValue(value);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -26,7 +26,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -75,9 +74,8 @@ public class TermJsonFilterParser extends AbstractIndexComponent implements Json
|
||||
Filter filter = null;
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
filter = fieldMapper.fieldFilter(value);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
filter = smartNameFieldMappers.mapper().fieldFilter(value);
|
||||
}
|
||||
}
|
||||
if (filter == null) {
|
||||
|
@ -27,7 +27,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -92,9 +91,8 @@ public class TermJsonQueryParser extends AbstractIndexComponent implements JsonQ
|
||||
Query query = null;
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
query = fieldMapper.fieldQuery(value);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
query = smartNameFieldMappers.mapper().fieldQuery(value);
|
||||
}
|
||||
}
|
||||
if (query == null) {
|
||||
|
@ -65,8 +65,8 @@ public class TermsJsonFilterParser extends AbstractIndexComponent implements Jso
|
||||
FieldMapper fieldMapper = null;
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
fieldMapper = smartNameFieldMappers.mapper();
|
||||
fieldName = fieldMapper.indexName();
|
||||
}
|
||||
}
|
||||
|
@ -28,7 +28,6 @@ import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonToken;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.FieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryParsingException;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
@ -91,10 +90,9 @@ public class WildcardJsonQueryParser extends AbstractIndexComponent implements J
|
||||
|
||||
MapperService.SmartNameFieldMappers smartNameFieldMappers = parseContext.smartFieldMappers(fieldName);
|
||||
if (smartNameFieldMappers != null) {
|
||||
FieldMapper fieldMapper = smartNameFieldMappers.fieldMappers().mapper();
|
||||
if (fieldMapper != null) {
|
||||
fieldName = fieldMapper.indexName();
|
||||
value = fieldMapper.indexedValue(value);
|
||||
if (smartNameFieldMappers.hasMapper()) {
|
||||
fieldName = smartNameFieldMappers.mapper().indexName();
|
||||
value = smartNameFieldMappers.mapper().indexedValue(value);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -59,14 +59,14 @@ public class SearchPhaseController {
|
||||
|
||||
public AggregatedDfs aggregateDfs(Iterable<DfsSearchResult> results) {
|
||||
ExtTObjectIntHasMap<Term> dfMap = new ExtTObjectIntHasMap<Term>().defaultReturnValue(-1);
|
||||
int numDocs = 0;
|
||||
long aggMaxDoc = 0;
|
||||
for (DfsSearchResult result : results) {
|
||||
for (int i = 0; i < result.freqs().length; i++) {
|
||||
dfMap.adjustOrPutValue(result.terms()[i], result.freqs()[i], result.freqs()[i]);
|
||||
}
|
||||
numDocs += result.numDocs();
|
||||
aggMaxDoc += result.maxDoc();
|
||||
}
|
||||
return new AggregatedDfs(dfMap, numDocs);
|
||||
return new AggregatedDfs(dfMap, aggMaxDoc);
|
||||
}
|
||||
|
||||
public ShardDoc[] sortDocs(Collection<? extends QuerySearchResultProvider> results) {
|
||||
|
@ -35,23 +35,23 @@ public class AggregatedDfs implements Streamable {
|
||||
|
||||
private ExtTObjectIntHasMap<Term> dfMap;
|
||||
|
||||
private int numDocs;
|
||||
private long maxDoc;
|
||||
|
||||
private AggregatedDfs() {
|
||||
|
||||
}
|
||||
|
||||
public AggregatedDfs(ExtTObjectIntHasMap<Term> dfMap, int numDocs) {
|
||||
public AggregatedDfs(ExtTObjectIntHasMap<Term> dfMap, long maxDoc) {
|
||||
this.dfMap = dfMap.defaultReturnValue(-1);
|
||||
this.numDocs = numDocs;
|
||||
this.maxDoc = maxDoc;
|
||||
}
|
||||
|
||||
public ExtTObjectIntHasMap<Term> dfMap() {
|
||||
return dfMap;
|
||||
}
|
||||
|
||||
public int numDocs() {
|
||||
return numDocs;
|
||||
public long maxDoc() {
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
public static AggregatedDfs readAggregatedDfs(DataInput in) throws IOException, ClassNotFoundException {
|
||||
@ -66,7 +66,7 @@ public class AggregatedDfs implements Streamable {
|
||||
for (int i = 0; i < size; i++) {
|
||||
dfMap.put(new Term(in.readUTF(), in.readUTF()), in.readInt());
|
||||
}
|
||||
numDocs = in.readInt();
|
||||
maxDoc = in.readLong();
|
||||
}
|
||||
|
||||
@Override public void writeTo(final DataOutput out) throws IOException {
|
||||
@ -75,7 +75,7 @@ public class AggregatedDfs implements Streamable {
|
||||
if (!dfMap.forEachEntry(writeToProcedure)) {
|
||||
throw writeToProcedure.exception;
|
||||
}
|
||||
out.writeInt(numDocs);
|
||||
out.writeLong(maxDoc);
|
||||
}
|
||||
|
||||
private static class WriteToProcedure implements TObjectIntProcedure<Term> {
|
||||
|
@ -33,9 +33,16 @@ public class CachedDfSource extends Searcher {
|
||||
|
||||
private final AggregatedDfs dfs;
|
||||
|
||||
private final int maxDoc;
|
||||
|
||||
public CachedDfSource(AggregatedDfs dfs, Similarity similarity) throws IOException {
|
||||
this.dfs = dfs;
|
||||
setSimilarity(similarity);
|
||||
if (dfs.maxDoc() > Integer.MAX_VALUE) {
|
||||
maxDoc = Integer.MAX_VALUE;
|
||||
} else {
|
||||
maxDoc = (int) dfs.maxDoc();
|
||||
}
|
||||
}
|
||||
|
||||
public int docFreq(Term term) {
|
||||
@ -55,7 +62,7 @@ public class CachedDfSource extends Searcher {
|
||||
}
|
||||
|
||||
public int maxDoc() {
|
||||
return dfs.numDocs();
|
||||
return this.maxDoc;
|
||||
}
|
||||
|
||||
public Query rewrite(Query query) {
|
||||
|
@ -47,7 +47,7 @@ public class DfsPhase implements SearchPhase {
|
||||
int[] freqs = context.searcher().docFreqs(terms);
|
||||
|
||||
context.dfsResult().termsAndFreqs(terms, freqs);
|
||||
context.dfsResult().numDocs(context.searcher().getIndexReader().numDocs());
|
||||
context.dfsResult().maxDoc(context.searcher().getIndexReader().maxDoc());
|
||||
} catch (Exception e) {
|
||||
throw new DfsPhaseExecutionException(context);
|
||||
}
|
||||
|
@ -46,7 +46,7 @@ public class DfsSearchResult implements Streamable {
|
||||
|
||||
private int[] freqs;
|
||||
|
||||
private int numDocs;
|
||||
private int maxDoc;
|
||||
|
||||
public DfsSearchResult() {
|
||||
|
||||
@ -65,13 +65,13 @@ public class DfsSearchResult implements Streamable {
|
||||
return shardTarget;
|
||||
}
|
||||
|
||||
public DfsSearchResult numDocs(int numDocs) {
|
||||
this.numDocs = numDocs;
|
||||
public DfsSearchResult maxDoc(int maxDoc) {
|
||||
this.maxDoc = maxDoc;
|
||||
return this;
|
||||
}
|
||||
|
||||
public int numDocs() {
|
||||
return numDocs;
|
||||
public int maxDoc() {
|
||||
return maxDoc;
|
||||
}
|
||||
|
||||
public DfsSearchResult termsAndFreqs(Term[] terms, int[] freqs) {
|
||||
@ -115,7 +115,7 @@ public class DfsSearchResult implements Streamable {
|
||||
freqs[i] = in.readInt();
|
||||
}
|
||||
}
|
||||
numDocs = in.readInt();
|
||||
maxDoc = in.readInt();
|
||||
}
|
||||
|
||||
@Override public void writeTo(DataOutput out) throws IOException {
|
||||
@ -130,6 +130,6 @@ public class DfsSearchResult implements Streamable {
|
||||
for (int freq : freqs) {
|
||||
out.writeInt(freq);
|
||||
}
|
||||
out.writeInt(numDocs);
|
||||
out.writeInt(maxDoc);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,59 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
*/
|
||||
public class BoundedTreeSet<E> extends TreeSet<E> {
|
||||
|
||||
private final int size;
|
||||
|
||||
public BoundedTreeSet(int size) {
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public BoundedTreeSet(Comparator<? super E> comparator, int size) {
|
||||
super(comparator);
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
@Override public boolean add(E e) {
|
||||
boolean result = super.add(e);
|
||||
rebound();
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override public boolean addAll(Collection<? extends E> c) {
|
||||
boolean result = super.addAll(c);
|
||||
rebound();
|
||||
return result;
|
||||
}
|
||||
|
||||
private void rebound() {
|
||||
while (size() > size) {
|
||||
remove(last());
|
||||
}
|
||||
}
|
||||
}
|
@ -20,8 +20,10 @@
|
||||
package org.elasticsearch.deps.lucene;
|
||||
|
||||
import org.apache.lucene.document.*;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermEnum;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
@ -31,6 +33,7 @@ import org.apache.lucene.util.NumericUtils;
|
||||
import org.elasticsearch.util.lucene.Lucene;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import static org.elasticsearch.util.lucene.DocumentBuilder.*;
|
||||
@ -120,4 +123,91 @@ public class SimpleLuceneTests {
|
||||
|
||||
indexWriter.close();
|
||||
}
|
||||
|
||||
@Test public void testTermEnumDocFreq() throws Exception {
|
||||
Directory dir = new RAMDirectory();
|
||||
IndexWriter indexWriter = new IndexWriter(dir, Lucene.STANDARD_ANALYZER, true, IndexWriter.MaxFieldLength.UNLIMITED);
|
||||
|
||||
IndexReader reader = indexWriter.getReader();
|
||||
|
||||
Document doc = new Document();
|
||||
doc.add(new Field("id", "1", Field.Store.NO, Field.Index.ANALYZED));
|
||||
doc.add(new Field("value", "aaa", Field.Store.NO, Field.Index.ANALYZED));
|
||||
indexWriter.addDocument(doc);
|
||||
|
||||
reader = refreshReader(reader);
|
||||
|
||||
TermEnum termEnum = reader.terms(new Term("value", ""));
|
||||
assertThat(termEnum.term().text(), equalTo("aaa"));
|
||||
assertThat(termEnum.docFreq(), equalTo(1));
|
||||
termEnum.close();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new Field("id", "2", Field.Store.NO, Field.Index.ANALYZED));
|
||||
doc.add(new Field("value", "bbb bbb", Field.Store.NO, Field.Index.ANALYZED));
|
||||
indexWriter.addDocument(doc);
|
||||
|
||||
reader = refreshReader(reader);
|
||||
|
||||
termEnum = reader.terms(new Term("value", ""));
|
||||
assertThat(termEnum.term().text(), equalTo("aaa"));
|
||||
assertThat(termEnum.docFreq(), equalTo(1));
|
||||
termEnum.next();
|
||||
assertThat(termEnum.term().text(), equalTo("bbb"));
|
||||
assertThat(termEnum.docFreq(), equalTo(1));
|
||||
termEnum.close();
|
||||
|
||||
doc = new Document();
|
||||
doc.add(new Field("id", "3", Field.Store.NO, Field.Index.ANALYZED));
|
||||
doc.add(new Field("value", "bbb", Field.Store.NO, Field.Index.ANALYZED));
|
||||
indexWriter.addDocument(doc);
|
||||
|
||||
reader = refreshReader(reader);
|
||||
|
||||
termEnum = reader.terms(new Term("value", ""));
|
||||
assertThat(termEnum.term().text(), equalTo("aaa"));
|
||||
assertThat(termEnum.docFreq(), equalTo(1));
|
||||
termEnum.next();
|
||||
assertThat(termEnum.term().text(), equalTo("bbb"));
|
||||
assertThat(termEnum.docFreq(), equalTo(2));
|
||||
termEnum.close();
|
||||
|
||||
indexWriter.deleteDocuments(new Term("id", "3"));
|
||||
|
||||
reader = refreshReader(reader);
|
||||
|
||||
// won't see the changes until optimize
|
||||
termEnum = reader.terms(new Term("value", ""));
|
||||
assertThat(termEnum.term().text(), equalTo("aaa"));
|
||||
assertThat(termEnum.docFreq(), equalTo(1));
|
||||
termEnum.next();
|
||||
assertThat(termEnum.term().text(), equalTo("bbb"));
|
||||
assertThat(termEnum.docFreq(), equalTo(2));
|
||||
termEnum.close();
|
||||
|
||||
indexWriter.expungeDeletes();
|
||||
|
||||
reader = refreshReader(reader);
|
||||
|
||||
termEnum = reader.terms(new Term("value", ""));
|
||||
assertThat(termEnum.term().text(), equalTo("aaa"));
|
||||
assertThat(termEnum.docFreq(), equalTo(1));
|
||||
termEnum.next();
|
||||
assertThat(termEnum.term().text(), equalTo("bbb"));
|
||||
assertThat(termEnum.docFreq(), equalTo(1));
|
||||
termEnum.close();
|
||||
|
||||
|
||||
reader.close();
|
||||
indexWriter.close();
|
||||
}
|
||||
|
||||
private IndexReader refreshReader(IndexReader reader) throws IOException {
|
||||
IndexReader oldReader = reader;
|
||||
reader = reader.reopen();
|
||||
if (reader != oldReader) {
|
||||
oldReader.close();
|
||||
}
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
|
@ -101,7 +101,7 @@ public class SimpleSingleTransportClientTests extends AbstractServersTests {
|
||||
assertThat(indicesStatusResponse.failedShards(), equalTo(5)); // 5 are not active, since we started just one server
|
||||
assertThat(indicesStatusResponse.indices().size(), equalTo(1));
|
||||
assertThat(indicesStatusResponse.index("test").shards().size(), equalTo(5)); // 5 index shards (1 with 1 backup)
|
||||
assertThat(indicesStatusResponse.index("test").docs().numDocs(), equalTo(1));
|
||||
assertThat(indicesStatusResponse.index("test").docs().numDocs(), equalTo(1l));
|
||||
|
||||
GetResponse getResult;
|
||||
|
||||
|
@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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.server.internal.InternalServer;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.util.transport.TransportAddress;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
*/
|
||||
@Test
|
||||
public class ClientTransportTermsActionTests extends TermsActionTests {
|
||||
|
||||
@Override protected Client getClient() {
|
||||
TransportAddress server1Address = ((InternalServer) server("server1")).injector().getInstance(TransportService.class).boundAddress().publishAddress();
|
||||
TransportClient client = new TransportClient();
|
||||
client.addTransportAddress(server1Address);
|
||||
return client;
|
||||
}
|
||||
}
|
@ -0,0 +1,6 @@
|
||||
cluster:
|
||||
routing:
|
||||
schedule: 200ms
|
||||
index:
|
||||
numberOfShards: 5
|
||||
numberOfReplicas: 1
|
@ -0,0 +1,237 @@
|
||||
/*
|
||||
* 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.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.AbstractServersTests;
|
||||
import org.testng.annotations.AfterMethod;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import static org.elasticsearch.client.Requests.*;
|
||||
import static org.elasticsearch.util.json.JsonBuilder.*;
|
||||
import static org.hamcrest.MatcherAssert.*;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
/**
|
||||
* @author kimchy (Shay Banon)
|
||||
*/
|
||||
@Test
|
||||
public class TermsActionTests extends AbstractServersTests {
|
||||
|
||||
@AfterMethod public void closeServers() {
|
||||
closeAllServers();
|
||||
}
|
||||
|
||||
@Test public void testTermsAction() throws Exception {
|
||||
startServer("server1");
|
||||
startServer("server2");
|
||||
Client client = getClient();
|
||||
try {
|
||||
verifyTermsActions(client);
|
||||
} finally {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
|
||||
protected Client getClient() {
|
||||
return client("server2");
|
||||
}
|
||||
|
||||
protected void verifyTermsActions(Client client) throws Exception {
|
||||
logger.info("Creating index test");
|
||||
client.admin().indices().create(createIndexRequest("test")).actionGet();
|
||||
Thread.sleep(500);
|
||||
|
||||
IndexStatus indexStatus = client.admin().indices().status(indicesStatus("test")).actionGet().index("test");
|
||||
|
||||
// verify no freqs
|
||||
logger.info("Verify no freqs");
|
||||
TermsResponse termsResponse = client.terms(termsRequest("test").fields("value")).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("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].term(), equalTo("aaa"));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[1].term(), 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(TermsRequest.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].term(), equalTo("bbb"));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(2));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[1].term(), 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(TermsRequest.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].term(), 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(TermsRequest.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].term(), 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(TermsRequest.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].term(), 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].term(), equalTo("aaa"));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[1].term(), 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].term(), equalTo("aaa"));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[1].term(), 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].term(), equalTo("aaa"));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[0].docFreq(), equalTo(1));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[1].term(), equalTo("bbb"));
|
||||
assertThat(termsResponse.field("value").termsFreqs()[1].docFreq(), equalTo(1));
|
||||
}
|
||||
}
|
@ -0,0 +1,6 @@
|
||||
cluster:
|
||||
routing:
|
||||
schedule: 200ms
|
||||
index:
|
||||
numberOfShards: 5
|
||||
numberOfReplicas: 1
|
Loading…
x
Reference in New Issue
Block a user