Upgrade to Lucene 4.3.0

This Lucene Release introduced a new API on DocIdSetIterator that requires each
implementation to return a `cost` upperbound as a function of the iterated documents.
This API allows for several optimizations during query execution especially in
Conjunction and Disjunction Queries with min_should_match set.

Closes #2990
This commit is contained in:
Simon Willnauer 2013-04-17 22:12:07 +02:00
parent f566527513
commit 2219925485
43 changed files with 297 additions and 471 deletions

View File

@ -30,7 +30,7 @@
</parent>
<properties>
<lucene.version>4.2.1</lucene.version>
<lucene.version>4.3.0</lucene.version>
</properties>
<repositories>

View File

@ -1,71 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.lucene.analysis.miscellaneous;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import java.io.IOException;
/**
* This TokenFilter emits each incoming token twice once as keyword and once non-keyword, in other words once with
* {@link KeywordAttribute#setKeyword(boolean)} set to <code>true</code> and once set to <code>false</code>.
* This is useful if used with a stem filter that respects the {@link KeywordAttribute} to index the stemmed and the
* un-stemmed version of a term into the same field.
*/
//LUCENE MONITOR - this will be included in Lucene 4.3. (it's a plain copy of the lucene version)
public final class KeywordRepeatFilter extends TokenFilter {
private final KeywordAttribute keywordAttribute = addAttribute(KeywordAttribute.class);
private final PositionIncrementAttribute posIncAttr = addAttribute(PositionIncrementAttribute.class);
private State state;
/**
* Construct a token stream filtering the given input.
*/
public KeywordRepeatFilter(TokenStream input) {
super(input);
}
@Override
public boolean incrementToken() throws IOException {
if (state != null) {
restoreState(state);
posIncAttr.setPositionIncrement(0);
keywordAttribute.setKeyword(false);
state = null;
return true;
}
if (input.incrementToken()) {
state = captureState();
keywordAttribute.setKeyword(true);
return true;
}
return false;
}
@Override
public void reset() throws IOException {
super.reset();
state = null;
}
}

View File

@ -1,169 +0,0 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.lucene.analysis.miscellaneous;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.CharsRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.UnicodeUtil;
import org.apache.lucene.util.Version;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.FST.Arc;
import org.apache.lucene.util.fst.FST.BytesReader;
import org.elasticsearch.common.lucene.Lucene;
/**
* Provides the ability to override any {@link KeywordAttribute} aware stemmer
* with custom dictionary-based stemming.
*/
// LUCENE UPGRADE - this is a copy of the StemmerOverrideFilter from Lucene - update with 4.3
public final class XStemmerOverrideFilter extends TokenFilter {
private final FST<BytesRef> fst;
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final KeywordAttribute keywordAtt = addAttribute(KeywordAttribute.class);
private final BytesReader fstReader;
private final Arc<BytesRef> scratchArc = new FST.Arc<BytesRef>();
private final CharsRef spare = new CharsRef();
private final boolean ignoreCase;
static {
assert Version.LUCENE_42 == Lucene.VERSION: "Elasticsearch has upgraded to Lucene Version: [" + Lucene.VERSION + "] this should can be removed";
}
/**
* Create a new StemmerOverrideFilter, performing dictionary-based stemming
* with the provided <code>dictionary</code>.
* <p>
* Any dictionary-stemmed terms will be marked with {@link KeywordAttribute}
* so that they will not be stemmed with stemmers down the chain.
* </p>
*/
public XStemmerOverrideFilter(TokenStream input, StemmerOverrideMap stemmerOverrideMap, boolean ignoreCase) {
super(input);
this.fst = stemmerOverrideMap.fst;
fstReader = fst.getBytesReader();
this.ignoreCase = ignoreCase;
}
@Override
public boolean incrementToken() throws IOException {
if (input.incrementToken()) {
if (!keywordAtt.isKeyword()) { // don't muck with already-keyworded terms
final BytesRef stem = getStem(termAtt.buffer(), termAtt.length());
if (stem != null) {
final char[] buffer = spare.chars = termAtt.buffer();
UnicodeUtil.UTF8toUTF16(stem.bytes, stem.offset, stem.length, spare);
if (spare.chars != buffer) {
termAtt.copyBuffer(spare.chars, spare.offset, spare.length);
}
termAtt.setLength(spare.length);
keywordAtt.setKeyword(true);
}
}
return true;
} else {
return false;
}
}
private BytesRef getStem(char[] buffer, int bufferLen) throws IOException {
BytesRef pendingOutput = fst.outputs.getNoOutput();
BytesRef matchOutput = null;
int bufUpto = 0;
fst.getFirstArc(scratchArc);
while (bufUpto < bufferLen) {
final int codePoint = Character.codePointAt(buffer, bufUpto, bufferLen);
if (fst.findTargetArc(ignoreCase ? Character.toLowerCase(codePoint) : codePoint, scratchArc, scratchArc, fstReader) == null) {
return null;
}
pendingOutput = fst.outputs.add(pendingOutput, scratchArc.output);
bufUpto += Character.charCount(codePoint);
}
if (scratchArc.isFinal()) {
matchOutput = fst.outputs.add(pendingOutput, scratchArc.nextFinalOutput);
}
return matchOutput;
}
public static class StemmerOverrideMap {
final FST<BytesRef> fst;
StemmerOverrideMap(FST<BytesRef> fst) {
this.fst = fst;
}
}
/**
* This builder builds an {@link FST} for the {@link StemmerOverrideFilter}
*/
public static class Builder {
private final BytesRefHash hash = new BytesRefHash();
private final BytesRef spare = new BytesRef();
private final ArrayList<CharSequence> outputValues = new ArrayList<CharSequence>();
/**
* Adds an input string and it's stemmer overwrite output to this builder.
*
* @param input the input char sequence
* @param output the stemmer override output char sequence
* @return <code>false</code> iff the input has already been added to this builder otherwise <code>true</code>.
*/
public boolean add(CharSequence input, CharSequence output) {
UnicodeUtil.UTF16toUTF8(input, 0, input.length(), spare);
int id = hash.add(spare);
if (id >= 0) {
outputValues.add(output);
return true;
}
return false;
}
/**
* Returns an {@link StemmerOverrideMap} to be used with the {@link StemmerOverrideFilter}
* @return an {@link StemmerOverrideMap} to be used with the {@link StemmerOverrideFilter}
* @throws IOException if an {@link IOException} occurs;
*/
public StemmerOverrideMap build() throws IOException {
ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
org.apache.lucene.util.fst.Builder<BytesRef> builder = new org.apache.lucene.util.fst.Builder<BytesRef>(
FST.INPUT_TYPE.BYTE4, outputs);
final int[] sort = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
IntsRef intsSpare = new IntsRef();
final int size = hash.size();
for (int i = 0; i < size; i++) {
int id = sort[i];
BytesRef bytesRef = hash.get(id, spare);
UnicodeUtil.UTF8toUTF32(bytesRef, intsSpare);
builder.add(intsSpare, new BytesRef(outputValues.get(id)));
}
return new StemmerOverrideMap(builder.finish());
}
}
}

View File

@ -106,4 +106,11 @@ public class TrackingConcurrentMergeScheduler extends ConcurrentMergeScheduler {
}
}
}
@Override
public MergeScheduler clone() {
// Lucene IW makes a clone internally but since we hold on to this instance
// the clone will just be the identity.
return this;
}
}

View File

@ -123,4 +123,11 @@ public class TrackingSerialMergeScheduler extends MergeScheduler {
@Override
public void close() {
}
@Override
public MergeScheduler clone() {
// Lucene IW makes a clone internally but since we hold on to this instance
// the clone will just be the identity.
return this;
}
}

View File

@ -1,168 +0,0 @@
package org.apache.lucene.search.vectorhighlight;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF 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.
*/
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import org.apache.lucene.search.vectorhighlight.FieldPhraseList.WeightedPhraseInfo;
import org.apache.lucene.util.Version;
import org.elasticsearch.common.lucene.Lucene;
/**
* A non-abstract copy of the abstract {@link BaseFragListBuilder}. that works
* in the same way as {@link SimpleFragListBuilder} but isn't prone to the
* problem of negative offsets. This is fixed in Lucene 4.3 and this class
* should be removed once Elasticsearch upgraded to Lucene 4.3
* <p>
* LUCENE-4899: FastVectorHighlihgter failed with
* {@link StringIndexOutOfBoundsException} if a single highlight phrase or term was
* greater than the fragCharSize producing negative string offsets
* </p>
*/
public final class XSimpleFragListBuilder implements FragListBuilder {
public static final int MARGIN_DEFAULT = 6;
public static final int MIN_FRAG_CHAR_SIZE_FACTOR = 3;
final int margin;
final int minFragCharSize;
static {
assert Version.LUCENE_42 == Lucene.VERSION: "Elasticsearch has upgraded to Lucene Version: [" + Lucene.VERSION + "] this should can be removed";
}
public XSimpleFragListBuilder(int margin) {
if (margin < 0)
throw new IllegalArgumentException("margin(" + margin + ") is too small. It must be 0 or higher.");
this.margin = margin;
this.minFragCharSize = Math.max(1, margin * MIN_FRAG_CHAR_SIZE_FACTOR);
}
public XSimpleFragListBuilder() {
this(MARGIN_DEFAULT);
}
@Override
public FieldFragList createFieldFragList(FieldPhraseList fieldPhraseList, int fragCharSize) {
return createFieldFragList(fieldPhraseList, new SimpleFieldFragList(fragCharSize), fragCharSize);
}
protected FieldFragList createFieldFragList(FieldPhraseList fieldPhraseList, FieldFragList fieldFragList, int fragCharSize) {
if (fragCharSize < minFragCharSize)
throw new IllegalArgumentException("fragCharSize(" + fragCharSize + ") is too small. It must be " + minFragCharSize + " or higher.");
List<WeightedPhraseInfo> wpil = new ArrayList<WeightedPhraseInfo>();
IteratorQueue<WeightedPhraseInfo> queue = new IteratorQueue<WeightedPhraseInfo>(fieldPhraseList.getPhraseList().iterator());
WeightedPhraseInfo phraseInfo = null;
int startOffset = 0;
while ((phraseInfo = queue.top()) != null) {
// if the phrase violates the border of previous fragment, discard
// it and try next phrase
if (phraseInfo.getStartOffset() < startOffset) {
queue.removeTop();
continue;
}
wpil.clear();
final int currentPhraseStartOffset = phraseInfo.getStartOffset();
int currentPhraseEndOffset = phraseInfo.getEndOffset();
int spanStart = Math.max(currentPhraseStartOffset - margin, startOffset);
int spanEnd = Math.max(currentPhraseEndOffset, spanStart + fragCharSize);
if (acceptPhrase(queue.removeTop(), currentPhraseEndOffset - currentPhraseStartOffset, fragCharSize)) {
wpil.add(phraseInfo);
}
while ((phraseInfo = queue.top()) != null) { // pull until we crossed the current spanEnd
if (phraseInfo.getEndOffset() <= spanEnd) {
currentPhraseEndOffset = phraseInfo.getEndOffset();
if (acceptPhrase(queue.removeTop(), currentPhraseEndOffset - currentPhraseStartOffset, fragCharSize)) {
wpil.add(phraseInfo);
}
} else {
break;
}
}
if (wpil.isEmpty()) {
continue;
}
final int matchLen = currentPhraseEndOffset - currentPhraseStartOffset;
// now recalculate the start and end position to "center" the result
final int newMargin = Math.max(0, (fragCharSize - matchLen) / 2);
// matchLen can be > fragCharSize prevent IAOOB here
spanStart = currentPhraseStartOffset - newMargin;
if (spanStart < startOffset) {
spanStart = startOffset;
}
// whatever is bigger here we grow this out
spanEnd = spanStart + Math.max(matchLen, fragCharSize);
startOffset = spanEnd;
fieldFragList.add(spanStart, spanEnd, wpil);
}
return fieldFragList;
}
/**
* A predicate to decide if the given {@link WeightedPhraseInfo} should be
* accepted as a highlighted phrase or if it should be discarded.
* <p>
* The default implementation discards phrases that are composed of more
* than one term and where the matchLength exceeds the fragment character
* size.
*
* @param info
* the phrase info to accept
* @param matchLength
* the match length of the current phrase
* @param fragCharSize
* the configured fragment character size
* @return <code>true</code> if this phrase info should be accepted as a
* highligh phrase
*/
protected boolean acceptPhrase(WeightedPhraseInfo info, int matchLength, int fragCharSize) {
return info.getTermsOffsets().size() <= 1 || matchLength <= fragCharSize;
}
private static final class IteratorQueue<T> {
private final Iterator<T> iter;
private T top;
public IteratorQueue(Iterator<T> iter) {
this.iter = iter;
T removeTop = removeTop();
assert removeTop == null;
}
public T top() {
return top;
}
public T removeTop() {
T currentTop = top;
if (iter.hasNext()) {
top = iter.next();
} else {
top = null;
}
return currentTop;
}
}
}

View File

@ -121,10 +121,10 @@ public class Version implements Serializable {
public static final int V_0_90_0_ID = /*00*/900099;
public static final Version V_0_90_0 = new Version(V_0_90_0_ID, false, org.apache.lucene.util.Version.LUCENE_42);
public static final int V_0_90_1_ID = /*00*/900199;
public static final Version V_0_90_1 = new Version(V_0_90_1_ID, false, org.apache.lucene.util.Version.LUCENE_42);
public static final Version V_0_90_1 = new Version(V_0_90_1_ID, false, org.apache.lucene.util.Version.LUCENE_43);
public static final int V_1_0_0_Beta1_ID = /*00*/1000001;
public static final Version V_1_0_0_Beta1 = new Version(V_1_0_0_Beta1_ID, true, org.apache.lucene.util.Version.LUCENE_42);
public static final Version V_1_0_0_Beta1 = new Version(V_1_0_0_Beta1_ID, true, org.apache.lucene.util.Version.LUCENE_43);
public static final Version CURRENT = V_1_0_0_Beta1;

View File

@ -42,7 +42,7 @@ import java.lang.reflect.Field;
*/
public class Lucene {
public static final Version VERSION = Version.LUCENE_42;
public static final Version VERSION = Version.LUCENE_43;
public static final Version ANALYZER_VERSION = VERSION;
public static final Version QUERYPARSER_VERSION = VERSION;
@ -57,6 +57,9 @@ public class Lucene {
if (version == null) {
return defaultVersion;
}
if ("4.3".equals(version)) {
return Version.LUCENE_43;
}
if ("4.2".equals(version)) {
return Version.LUCENE_42;
}
@ -375,8 +378,8 @@ public class Lucene {
private Lucene() {
}
public static final boolean indexExists(final Directory directory) {
public static final boolean indexExists(final Directory directory) throws IOException {
return DirectoryReader.indexExists(directory);
}
}

View File

@ -84,5 +84,11 @@ public class AllDocIdSet extends DocIdSet {
}
return doc = NO_MORE_DOCS;
}
@Override
public long cost() {
return maxDoc;
}
}
}

View File

@ -119,23 +119,28 @@ public class AndDocIdSet extends DocIdSet {
static class IteratorBasedIterator extends DocIdSetIterator {
int lastReturn = -1;
private DocIdSetIterator[] iterators = null;
private final long cost;
IteratorBasedIterator(DocIdSet[] sets) throws IOException {
iterators = new DocIdSetIterator[sets.length];
int j = 0;
long cost = Integer.MAX_VALUE;
for (DocIdSet set : sets) {
if (set == null) {
lastReturn = DocIdSetIterator.NO_MORE_DOCS; // non matching
break;
} else {
DocIdSetIterator dcit = set.iterator();
if (dcit == null) {
lastReturn = DocIdSetIterator.NO_MORE_DOCS; // non matching
break;
}
iterators[j++] = dcit;
cost = Math.min(cost, dcit.cost());
}
}
this.cost = cost;
if (lastReturn != DocIdSetIterator.NO_MORE_DOCS) {
lastReturn = (iterators.length > 0 ? -1 : DocIdSetIterator.NO_MORE_DOCS);
}
@ -204,5 +209,10 @@ public class AndDocIdSet extends DocIdSet {
}
return (lastReturn = target);
}
@Override
public long cost() {
return cost;
}
}
}

View File

@ -59,4 +59,9 @@ public class BitsDocIdSetIterator extends MatchDocIdSetIterator {
return bits.get(doc);
}
}
@Override
public long cost() {
return this.bits.length();
}
}

View File

@ -106,6 +106,11 @@ public abstract class MatchDocIdSet extends DocIdSet implements Bits {
}
return doc = NO_MORE_DOCS;
}
@Override
public long cost() {
return maxDoc;
}
}
class FixedBitSetIterator extends FilteredDocIdSetIterator {
@ -155,5 +160,10 @@ public abstract class MatchDocIdSet extends DocIdSet implements Bits {
}
return doc = NO_MORE_DOCS;
}
@Override
public long cost() {
return maxDoc;
}
}
}

View File

@ -101,6 +101,11 @@ public class NotDocIdSet extends DocIdSet {
protected boolean matchDoc(int doc) {
return !bits.get(doc);
}
@Override
public long cost() {
return bits.length();
}
}
public static class IteratorBasedIterator extends DocIdSetIterator {
@ -157,5 +162,10 @@ public class NotDocIdSet extends DocIdSet {
return (lastReturn = target);
}
@Override
public long cost() {
return it1.cost();
}
}
}

View File

@ -101,17 +101,21 @@ public class OrDocIdSet extends DocIdSet {
private int _curDoc;
private final Item[] _heap;
private int _size;
private final long cost;
IteratorBasedIterator(DocIdSet[] sets) throws IOException {
_curDoc = -1;
_heap = new Item[sets.length];
_size = 0;
long cost = 0;
for (DocIdSet set : sets) {
DocIdSetIterator iterator = set.iterator();
if (iterator != null) {
_heap[_size++] = new Item(iterator);
cost += iterator.cost();
}
}
this.cost = cost;
if (_size == 0) _curDoc = DocIdSetIterator.NO_MORE_DOCS;
}
@ -228,5 +232,10 @@ public class OrDocIdSet extends DocIdSet {
}
}
@Override
public long cost() {
return cost;
}
}
}

View File

@ -45,7 +45,7 @@ public class EmptyScorer extends Scorer {
@Override
public int docID() {
return NO_MORE_DOCS;
return -1;
}
@Override
@ -57,4 +57,9 @@ public class EmptyScorer extends Scorer {
public int advance(int target) throws IOException {
return NO_MORE_DOCS;
}
@Override
public long cost() {
return 0;
}
}

View File

@ -347,6 +347,11 @@ public class FiltersFunctionScoreQuery extends Query {
public int freq() throws IOException {
return scorer.freq();
}
@Override
public long cost() {
return scorer.cost();
}
}

View File

@ -159,6 +159,11 @@ public class FunctionScoreQuery extends Query {
public int freq() throws IOException {
return scorer.freq();
}
@Override
public long cost() {
return scorer.cost();
}
}

View File

@ -19,20 +19,15 @@
package org.elasticsearch.index.analysis;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterators;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.br.BrazilianStemFilter;
import org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilter;
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.assistedinject.Assisted;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.settings.IndexSettings;
import java.util.Set;
/**
*
*/
@ -48,6 +43,6 @@ public class BrazilianStemTokenFilterFactory extends AbstractTokenFilterFactory
@Override
public TokenStream create(TokenStream tokenStream) {
return new BrazilianStemFilter(new KeywordMarkerFilter(tokenStream, exclusions));
return new BrazilianStemFilter(new SetKeywordMarkerFilter(tokenStream, exclusions));
}
}

View File

@ -20,7 +20,7 @@
package org.elasticsearch.index.analysis;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilter;
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
import org.apache.lucene.analysis.nl.DutchStemFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.elasticsearch.common.inject.Inject;
@ -44,6 +44,6 @@ public class DutchStemTokenFilterFactory extends AbstractTokenFilterFactory {
@Override
public TokenStream create(TokenStream tokenStream) {
return new DutchStemFilter(new KeywordMarkerFilter(tokenStream, exclusions));
return new DutchStemFilter(new SetKeywordMarkerFilter(tokenStream, exclusions));
}
}

View File

@ -19,11 +19,9 @@
package org.elasticsearch.index.analysis;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterators;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.fr.FrenchStemFilter;
import org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilter;
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.assistedinject.Assisted;
@ -46,6 +44,6 @@ public class FrenchStemTokenFilterFactory extends AbstractTokenFilterFactory {
@Override
public TokenStream create(TokenStream tokenStream) {
return new FrenchStemFilter(new KeywordMarkerFilter(tokenStream, exclusions));
return new FrenchStemFilter(new SetKeywordMarkerFilter(tokenStream, exclusions));
}
}

View File

@ -21,7 +21,7 @@ package org.elasticsearch.index.analysis;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.de.GermanStemFilter;
import org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilter;
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.inject.assistedinject.Assisted;
@ -44,6 +44,6 @@ public class GermanStemTokenFilterFactory extends AbstractTokenFilterFactory {
@Override
public TokenStream create(TokenStream tokenStream) {
return new GermanStemFilter(new KeywordMarkerFilter(tokenStream, exclusions));
return new GermanStemFilter(new SetKeywordMarkerFilter(tokenStream, exclusions));
}
}

View File

@ -19,8 +19,10 @@
package org.elasticsearch.index.analysis;
import java.util.Set;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.miscellaneous.KeywordMarkerFilter;
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.common.inject.Inject;
@ -30,8 +32,6 @@ import org.elasticsearch.env.Environment;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.settings.IndexSettings;
import java.util.Set;
@AnalysisSettingsRequired
public class KeywordMarkerTokenFilterFactory extends AbstractTokenFilterFactory {
@ -51,6 +51,6 @@ public class KeywordMarkerTokenFilterFactory extends AbstractTokenFilterFactory
@Override
public TokenStream create(TokenStream tokenStream) {
return new KeywordMarkerFilter(tokenStream, keywordLookup);
return new SetKeywordMarkerFilter(tokenStream, keywordLookup);
}
}

View File

@ -39,7 +39,7 @@ public abstract class NumericTokenizer extends Tokenizer {
}
protected NumericTokenizer(Reader reader, NumericTokenStream numericTokenStream, char[] buffer, Object extra) throws IOException {
super(numericTokenStream, reader);
super(reader);
this.numericTokenStream = numericTokenStream;
this.extra = extra;
this.buffer = buffer;

View File

@ -54,10 +54,6 @@ public class PatternTokenizerFactory extends AbstractTokenizerFactory {
@Override
public Tokenizer create(Reader reader) {
try {
return new PatternTokenizer(reader, pattern, group);
} catch (IOException e) {
throw new ElasticSearchIllegalStateException("failed to create pattern tokenizer", e);
}
return new PatternTokenizer(reader, pattern, group);
}
}

View File

@ -20,8 +20,8 @@
package org.elasticsearch.index.analysis;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.miscellaneous.XStemmerOverrideFilter;
import org.apache.lucene.analysis.miscellaneous.XStemmerOverrideFilter.StemmerOverrideMap;
import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter;
import org.apache.lucene.analysis.miscellaneous.StemmerOverrideFilter.StemmerOverrideMap;
import org.elasticsearch.ElasticSearchIllegalArgumentException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
@ -48,7 +48,7 @@ public class StemmerOverrideTokenFilterFactory extends AbstractTokenFilterFactor
throw new ElasticSearchIllegalArgumentException("stemmer override filter requires either `rules` or `rules_path` to be configured");
}
XStemmerOverrideFilter.Builder builder = new XStemmerOverrideFilter.Builder();
StemmerOverrideFilter.Builder builder = new StemmerOverrideFilter.Builder(false);
parseRules(rules, builder, "=>");
overrideMap = builder.build();
@ -56,10 +56,10 @@ public class StemmerOverrideTokenFilterFactory extends AbstractTokenFilterFactor
@Override
public TokenStream create(TokenStream tokenStream) {
return new XStemmerOverrideFilter(tokenStream, overrideMap, false);
return new StemmerOverrideFilter(tokenStream, overrideMap);
}
static void parseRules(List<String> rules, XStemmerOverrideFilter.Builder builder, String mappingSep) {
static void parseRules(List<String> rules, StemmerOverrideFilter.Builder builder, String mappingSep) {
for (String rule : rules) {
String key, override;
List<String> mapping = Strings.splitSmart(rule, mappingSep, false);

View File

@ -0,0 +1,72 @@
/*
* Licensed to ElasticSearch and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. ElasticSearch licenses this
* file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.deletionpolicy;
import org.apache.lucene.index.IndexDeletionPolicy;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.IndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
abstract class AbstractESDeletionPolicy extends IndexDeletionPolicy implements IndexShardComponent {
protected final ESLogger logger;
protected final ShardId shardId;
protected final Settings indexSettings;
protected final Settings componentSettings;
protected AbstractESDeletionPolicy(ShardId shardId, @IndexSettings Settings indexSettings) {
this.shardId = shardId;
this.indexSettings = indexSettings;
this.componentSettings = indexSettings.getComponentSettings(getClass());
this.logger = Loggers.getLogger(getClass(), indexSettings, shardId);
}
protected AbstractESDeletionPolicy(ShardId shardId, @IndexSettings Settings indexSettings, String prefixSettings) {
this.shardId = shardId;
this.indexSettings = indexSettings;
this.componentSettings = indexSettings.getComponentSettings(prefixSettings, getClass());
this.logger = Loggers.getLogger(getClass(), indexSettings, shardId);
}
@Override
public ShardId shardId() {
return this.shardId;
}
@Override
public Settings indexSettings() {
return this.indexSettings;
}
public String nodeName() {
return indexSettings.get("name", "");
}
}

View File

@ -33,7 +33,7 @@ import java.util.List;
/**
*
*/
public class KeepLastNDeletionPolicy extends AbstractIndexShardComponent implements IndexDeletionPolicy {
public class KeepLastNDeletionPolicy extends AbstractESDeletionPolicy {
private final int numToKeep;

View File

@ -20,11 +20,9 @@
package org.elasticsearch.index.deletionpolicy;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexDeletionPolicy;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import java.util.List;
@ -35,7 +33,7 @@ import java.util.List;
* all prior commits after a new commit is done. This is
* the default deletion policy.
*/
public class KeepOnlyLastDeletionPolicy extends AbstractIndexShardComponent implements IndexDeletionPolicy {
public class KeepOnlyLastDeletionPolicy extends AbstractESDeletionPolicy {
@Inject
public KeepOnlyLastDeletionPolicy(ShardId shardId, @IndexSettings Settings indexSettings) {

View File

@ -41,11 +41,11 @@ import java.util.concurrent.ConcurrentMap;
*
*
*/
public class SnapshotDeletionPolicy extends AbstractIndexShardComponent implements IndexDeletionPolicy {
public class SnapshotDeletionPolicy extends AbstractESDeletionPolicy {
private final IndexDeletionPolicy primary;
private ConcurrentMap<Long, SnapshotHolder> snapshots = ConcurrentCollections.newConcurrentMap();
private final ConcurrentMap<Long, SnapshotHolder> snapshots = ConcurrentCollections.newConcurrentMap();
private volatile List<SnapshotIndexCommit> commits;
@ -128,6 +128,13 @@ public class SnapshotDeletionPolicy extends AbstractIndexShardComponent implemen
}
}
@Override
public IndexDeletionPolicy clone() {
// Lucene IW makes a clone internally but since we hold on to this instance
// the clone will just be the identity. See RobinEngine recovery why we need this.
return this;
}
/**
* Helper method to snapshot a give commit.
*/

View File

@ -726,10 +726,15 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
}
@Override
public Searcher searcher() throws EngineException {
public Searcher searcher() throws EngineException{
SearcherManager manager = this.searcherManager;
IndexSearcher searcher = manager.acquire();
return new RobinSearcher(searcher, manager);
try {
IndexSearcher searcher = manager.acquire();
return new RobinSearcher(searcher, manager);
} catch (IOException ex) {
logger.error("failed to accquire searcher for shard [{}]", ex, shardId);
throw new EngineException(shardId, ex.getMessage());
}
}
@Override

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.merge.policy;
import org.apache.lucene.index.LogByteSizeMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.SegmentInfos;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.common.Preconditions;
@ -167,6 +168,13 @@ public class LogByteSizeMergePolicyProvider extends AbstractIndexShardComponent
super.close();
provider.policies.remove(this);
}
@Override
public MergePolicy clone() {
// Lucene IW makes a clone internally but since we hold on to this instance
// the clone will just be the identity.
return this;
}
}
public static class EnableMergeLogByteSizeMergePolicy extends CustomLogByteSizeMergePolicy {

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.merge.policy;
import org.apache.lucene.index.LogDocMergePolicy;
import org.apache.lucene.index.MergePolicy;
import org.apache.lucene.index.SegmentInfos;
import org.elasticsearch.ElasticSearchException;
import org.elasticsearch.common.Preconditions;
@ -168,5 +169,12 @@ public class LogDocMergePolicyProvider extends AbstractIndexShardComponent imple
}
return super.findMerges(trigger, infos);
}
@Override
public MergePolicy clone() {
// Lucene IW makes a clone internally but since we hold on to this instance
// the clone will just be the identity.
return this;
}
}
}

View File

@ -217,6 +217,13 @@ public class TieredMergePolicyProvider extends AbstractIndexShardComponent imple
super.close();
provider.policies.remove(this);
}
@Override
public MergePolicy clone() {
// Lucene IW makes a clone internally but since we hold on to this instance
// the clone will just be the identity.
return this;
}
}
public static class EnableMergeTieredMergePolicyProvider extends CustomTieredMergePolicyProvider {

View File

@ -301,6 +301,11 @@ public class ChildrenQuery extends Query implements SearchContext.Rewrite {
return nextDoc();
}
}
@Override
public long cost() {
return parentsIterator.cost();
}
}
static class AvgParentScorer extends ParentScorer {

View File

@ -307,5 +307,10 @@ public class ParentQuery extends Query implements SearchContext.Rewrite {
}
return currentChildDoc;
}
@Override
public long cost() {
return childrenIterator.cost();
}
}
}

View File

@ -386,5 +386,10 @@ public class TopChildrenQuery extends Query implements SearchContext.Rewrite {
public int freq() throws IOException {
return docs[index].count; // The number of matches in the child doc, which is propagated to parent
}
@Override
public long cost() {
return docs.length;
}
}
}

View File

@ -222,6 +222,11 @@ public class IncludeNestedDocsQuery extends Query {
public int docID() {
return currentDoc;
}
@Override
public long cost() {
return parentScorer.cost();
}
}
@Override

View File

@ -311,11 +311,7 @@ public class IndicesAnalysisService extends AbstractComponent {
@Override
public Tokenizer create(Reader reader) {
try {
return new PatternTokenizer(reader, Regex.compile("\\W+", null), -1);
} catch (IOException e) {
throw new ElasticSearchIllegalStateException("failed to parse default pattern");
}
return new PatternTokenizer(reader, Regex.compile("\\W+", null), -1);
}
}));

View File

@ -267,7 +267,7 @@ public class HighlightPhase extends AbstractComponent implements FetchSubPhase {
fragmentsBuilder = new SourceSimpleFragmentsBuilder(mapper, context, field.preTags(), field.postTags(), boundaryScanner);
}
} else {
fragListBuilder = field.fragmentOffset() == -1 ? new XSimpleFragListBuilder() : new XSimpleFragListBuilder(field.fragmentOffset());
fragListBuilder = field.fragmentOffset() == -1 ? new SimpleFragListBuilder() : new SimpleFragListBuilder(field.fragmentOffset());
if (field.scoreOrdered()) {
if (mapper.fieldType().stored()) {
fragmentsBuilder = new ScoreOrderFragmentsBuilder(field.preTags(), field.postTags(), boundaryScanner);

View File

@ -309,6 +309,11 @@ final class QueryRescorer implements Rescorer {
while(nextDoc() < target) {}
return docId;
}
@Override
public long cost() {
return docIds.length;
}
};
}
};

View File

@ -174,6 +174,46 @@ public class SimpleIndexStatsTests extends AbstractNodesTests {
assertThat(stats.getTotal().getSearch(), nullValue());
}
@Test
public void testMergeStats() {
client.admin().indices().prepareDelete().execute().actionGet();
// rely on 1 replica for this tests
client.admin().indices().prepareCreate("test1").execute().actionGet();
ClusterHealthResponse clusterHealthResponse = client.admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
// clear all
IndicesStatsResponse stats = client.admin().indices().prepareStats()
.setDocs(false)
.setStore(false)
.setIndexing(false)
.setFlush(true)
.setRefresh(true)
.setMerge(true)
.clear() // reset defaults
.execute().actionGet();
assertThat(stats.getTotal().getDocs(), nullValue());
assertThat(stats.getTotal().getStore(), nullValue());
assertThat(stats.getTotal().getIndexing(), nullValue());
assertThat(stats.getTotal().getGet(), nullValue());
assertThat(stats.getTotal().getSearch(), nullValue());
for (int i = 0; i < 20; i++) {
client.prepareIndex("test1", "type1", Integer.toString(i)).setSource("field", "value").execute().actionGet();
client.prepareIndex("test1", "type2", Integer.toString(i)).setSource("field", "value").execute().actionGet();
client.admin().indices().prepareFlush().execute().actionGet();
}
client.admin().indices().prepareOptimize().setWaitForMerge(true).setMaxNumSegments(1).execute().actionGet();
stats = client.admin().indices().prepareStats()
.setMerge(true)
.execute().actionGet();
assertThat(stats.getTotal().getMerge(), notNullValue());
assertThat(stats.getTotal().getMerge().getTotal(), greaterThan(0l));
}
@Test
public void testAllFlags() throws Exception {
client.admin().indices().prepareDelete().execute().actionGet();

View File

@ -50,7 +50,6 @@ import org.elasticsearch.common.geo.ShapeBuilder;
import org.elasticsearch.common.geo.ShapeBuilder.MultiPolygonBuilder;
import org.elasticsearch.common.geo.ShapeBuilder.PolygonBuilder;
import org.elasticsearch.common.io.Streams;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.FilterBuilders;
import org.elasticsearch.search.SearchHit;
@ -58,8 +57,6 @@ import org.elasticsearch.test.integration.AbstractNodesTests;
import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
import org.apache.lucene.spatial.prefix.tree.Node;
import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree;
import org.apache.lucene.spatial.query.SpatialArgs;
import org.apache.lucene.spatial.query.SpatialOperation;
import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;

View File

@ -19,8 +19,13 @@
package org.elasticsearch.test.unit.index.search.geo;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.not;
import org.apache.lucene.spatial.prefix.tree.Cell;
import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
import org.apache.lucene.spatial.prefix.tree.Node;
import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree;
import org.elasticsearch.common.geo.GeoPoint;
import org.elasticsearch.common.geo.GeoUtils;
@ -29,11 +34,6 @@ import org.testng.annotations.Test;
import com.spatial4j.core.context.SpatialContext;
import com.spatial4j.core.distance.DistanceUtils;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.not;
/**
*
*/
@ -214,7 +214,7 @@ public class GeoUtilsTests {
SpatialContext spatialContext = new SpatialContext(true);
GeohashPrefixTree geohashPrefixTree = new GeohashPrefixTree(spatialContext, GeohashPrefixTree.getMaxLevelsPossible()/2);
Node gNode = geohashPrefixTree.getWorldNode();
Cell gNode = geohashPrefixTree.getWorldCell();
for(int i = 0; i<geohashPrefixTree.getMaxLevels(); i++) {
double width = GeoUtils.geoHashCellWidth(i);
@ -234,7 +234,7 @@ public class GeoUtilsTests {
}
QuadPrefixTree quadPrefixTree = new QuadPrefixTree(spatialContext);
Node qNode = quadPrefixTree.getWorldNode();
Cell qNode = quadPrefixTree.getWorldCell();
for (int i = 0; i < QuadPrefixTree.DEFAULT_MAX_LEVELS; i++) {
double degrees = 360.0/(1L<<i);