Index: lucene/src/java/org/apache/lucene/search/AutomatonQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/AutomatonQuery.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/AutomatonQuery.java (working copy) @@ -24,6 +24,7 @@ import org.apache.lucene.index.TermsEnum; import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.ToStringUtils; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.ByteRunAutomaton; @@ -85,7 +86,7 @@ } @Override - protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { + protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException { // matches nothing if (BasicOperations.isEmpty(automaton)) { return TermsEnum.EMPTY; Index: lucene/src/java/org/apache/lucene/search/BooleanQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/BooleanQuery.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/BooleanQuery.java (working copy) @@ -368,6 +368,35 @@ return new BooleanWeight(searcher); } + // TODO: think about a better TermQuery reordering without instanceof checks and lots of special cases + + static final Term extractTerm(Query q) { + if (q instanceof ConstantScoreQuery) { + final Filter f = ((ConstantScoreQuery) q).getFilter(); + if (f instanceof QueryWrapperFilter) { + q = ((QueryWrapperFilter) f).getQuery(); + } + } + + return (q instanceof TermQuery) ? ((TermQuery) q).getTerm() : null; + } + + private static final Comparator tqComp = new Comparator() { + public int compare(BooleanClause one, BooleanClause two) { + Term t1 = extractTerm(one.getQuery()); + Term t2 = extractTerm(two.getQuery()); + if (t1 != null && t2 != null) { + return t1.compareTo(t2); + } else if (t1 != null) { + return -1; + } else if (t2 != null) { + return 1; + } else { + return 0; + } + } + }; + @Override public Query rewrite(IndexReader reader) throws IOException { if (minNrShouldMatch == 0 && clauses.size() == 1) { // optimize 1-clause queries @@ -386,20 +415,18 @@ } } - BooleanQuery clone = null; // recursively rewrite + BooleanQuery clone = (BooleanQuery) this.clone(); + for (int i = 0 ; i < clauses.size(); i++) { BooleanClause c = clauses.get(i); Query query = c.getQuery().rewrite(reader); if (query != c.getQuery()) { // clause rewrote: must clone - if (clone == null) - clone = (BooleanQuery)this.clone(); clone.clauses.set(i, new BooleanClause(query, c.getOccur())); } } - if (clone != null) { - return clone; // some clauses rewrote - } else - return this; // no clauses rewrote + + Collections.sort(clone.clauses, tqComp); + return clone.equals(this) ? this : clone; // some clauses rewrote } // inherit javadoc Index: lucene/src/java/org/apache/lucene/search/FuzzyQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/FuzzyQuery.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/FuzzyQuery.java (working copy) @@ -20,6 +20,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.ToStringUtils; import org.apache.lucene.util.automaton.LevenshteinAutomata; @@ -135,11 +136,11 @@ } @Override - protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { + protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException { if (!termLongEnough) { // can only match if it's exact return new SingleTermsEnum(reader, term); } - return new FuzzyTermsEnum(reader, getTerm(), minimumSimilarity, prefixLength); + return new FuzzyTermsEnum(reader, atts, getTerm(), minimumSimilarity, prefixLength); } /** Index: lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java =================================================================== --- lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (working copy) @@ -22,6 +22,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IntsRef; @@ -51,8 +52,13 @@ private final MultiTermQuery.BoostAttribute boostAtt = attributes().addAttribute(MultiTermQuery.BoostAttribute.class); - private float bottom = boostAtt.getMaxNonCompetitiveBoost(); + private final MultiTermQuery.MaxNonCompetitiveBoostAttribute maxBoostAtt; + private float bottom; + private BytesRef bottomTerm; + // nocommit: chicken-and-egg + private final Comparator termComparator = BytesRef.getUTF8SortedAsUnicodeComparator(); + private final float minSimilarity; private final float scale_factor; @@ -82,7 +88,7 @@ * @param prefixLength Length of required common prefix. Default value is 0. * @throws IOException */ - public FuzzyTermsEnum(IndexReader reader, Term term, + public FuzzyTermsEnum(IndexReader reader, AttributeSource atts, Term term, final float minSimilarity, final int prefixLength) throws IOException { if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity) throw new IllegalArgumentException("fractional edit distances are not allowed"); @@ -92,7 +98,9 @@ throw new IllegalArgumentException("prefixLength cannot be less than 0"); this.reader = reader; this.term = term; - + this.maxBoostAtt = atts.addAttribute(MultiTermQuery.MaxNonCompetitiveBoostAttribute.class); + bottom = maxBoostAtt.getMaxNonCompetitiveBoost(); + bottomTerm = maxBoostAtt.getCompetitiveTerm(); // convert the string into a utf32 int[] representation for fast comparisons final String utf16 = term.text(); this.termText = new int[utf16.codePointCount(0, utf16.length())]; @@ -116,9 +124,12 @@ } this.scale_factor = 1.0f / (1.0f - this.minSimilarity); + // TODO: merge this in with bottomChanged TermsEnum subEnum = getAutomatonEnum(maxEdits, null); setEnum(subEnum != null ? subEnum : new LinearFuzzyTermsEnum()); + if (bottom != Float.NEGATIVE_INFINITY) + bottomChanged(null); } /** @@ -169,13 +180,16 @@ * fired when the max non-competitive boost has changed. this is the hook to * swap in a smarter actualEnum */ - private void bottomChanged(float boostValue, BytesRef lastTerm) + private void bottomChanged(BytesRef lastTerm) throws IOException { int oldMaxEdits = maxEdits; + // true if the last term encountered is lexicographically equal or after the bottom term in the PQ + boolean termAfter = bottomTerm == null || (lastTerm != null && termComparator.compare(lastTerm, bottomTerm) >= 0); + // as long as the max non-competitive boost is >= the max boost // for some edit distance, keep dropping the max edit distance. - while (maxEdits > 0 && boostValue >= calculateMaxBoost(maxEdits)) + while (maxEdits > 0 && (termAfter ? bottom >= calculateMaxBoost(maxEdits) : bottom > calculateMaxBoost(maxEdits))) maxEdits--; if (oldMaxEdits != maxEdits) { // the maximum n has changed @@ -202,12 +216,14 @@ BytesRef term = actualEnum.next(); boostAtt.setBoost(actualBoostAtt.getBoost()); - final float bottom = boostAtt.getMaxNonCompetitiveBoost(); - if (bottom != this.bottom) { + final float bottom = maxBoostAtt.getMaxNonCompetitiveBoost(); + final BytesRef bottomTerm = maxBoostAtt.getCompetitiveTerm(); + if (bottom != this.bottom || bottomTerm != this.bottomTerm) { this.bottom = bottom; + this.bottomTerm = bottomTerm; // clone the term before potentially doing something with it // this is a rare but wonderful occurrence anyway - bottomChanged(bottom, term == null ? null : (BytesRef) term.clone()); + bottomChanged(term == null ? null : (BytesRef) term.clone()); } return term; Index: lucene/src/java/org/apache/lucene/search/MultiTermQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (working copy) @@ -19,19 +19,29 @@ import java.io.IOException; import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.PriorityQueue; +import java.util.Comparator; +import org.apache.lucene.index.Fields; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; -import org.apache.lucene.util.BytesRef; +import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; -import org.apache.lucene.index.MultiFields; -import org.apache.lucene.index.Fields; -import org.apache.lucene.index.Terms; -import org.apache.lucene.queryParser.QueryParser; // for javadoc +import org.apache.lucene.queryParser.QueryParser; +import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.Attribute; import org.apache.lucene.util.AttributeImpl; -import org.apache.lucene.util.PagedBytes; +import org.apache.lucene.util.AttributeSource; +import org.apache.lucene.util.ByteBlockPool; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefHash; +import org.apache.lucene.util.RamUsageEstimator; +import org.apache.lucene.util.ReaderUtil; +import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray; /** * An abstract {@link Query} that matches documents @@ -82,23 +92,11 @@ public void setBoost(float boost); /** Retrieves the boost, default is {@code 1.0f}. */ public float getBoost(); - /** Sets the maximum boost for terms that would never get - * into the priority queue of {@link MultiTermQuery.TopTermsBooleanQueryRewrite}. - * This value is not changed by {@link AttributeImpl#clear} - * and not used in {@code equals()} and {@code hashCode()}. - * Do not change the value in the {@link TermsEnum}! - */ - public void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost); - /** Retrieves the maximum boost that is not competitive, - * default is megative infinity. You can use this boost value - * as a hint when writing the {@link TermsEnum}. - */ - public float getMaxNonCompetitiveBoost(); } /** Implementation class for {@link BoostAttribute}. */ public static final class BoostAttributeImpl extends AttributeImpl implements BoostAttribute { - private float boost = 1.0f, maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY; + private float boost = 1.0f; public void setBoost(float boost) { this.boost = boost; @@ -107,8 +105,50 @@ public float getBoost() { return boost; } + + @Override + public void clear() { + boost = 1.0f; + } + + @Override + public boolean equals(Object other) { + if (this == other) + return true; + if (other instanceof BoostAttributeImpl) + return ((BoostAttributeImpl) other).boost == boost; + return false; + } + + @Override + public int hashCode() { + return Float.floatToIntBits(boost); + } + + @Override + public void copyTo(AttributeImpl target) { + ((BoostAttribute) target).setBoost(boost); + } + } + + /** TODO */ + public static interface MaxNonCompetitiveBoostAttribute extends Attribute { + /** TODO */ + public void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost); + /** TODO */ + public float getMaxNonCompetitiveBoost(); + /** TODO */ + public void setCompetitiveTerm(BytesRef competitiveTerm); + /** TODO */ + public BytesRef getCompetitiveTerm(); + } + + /** Implementation class for {@link MaxNonCompetitiveBoostAttribute}. */ + public static final class MaxNonCompetitiveBoostAttributeImpl extends AttributeImpl implements MaxNonCompetitiveBoostAttribute { + private float maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY; + private BytesRef competitiveTerm = null; - public void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost) { + public void setMaxNonCompetitiveBoost(final float maxNonCompetitiveBoost) { this.maxNonCompetitiveBoost = maxNonCompetitiveBoost; } @@ -116,28 +156,44 @@ return maxNonCompetitiveBoost; } + public void setCompetitiveTerm(final BytesRef competitiveTerm) { + this.competitiveTerm = competitiveTerm; + } + + public BytesRef getCompetitiveTerm() { + return competitiveTerm; + } + @Override public void clear() { - boost = 1.0f; + maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY; + competitiveTerm = null; } @Override public boolean equals(Object other) { if (this == other) return true; - if (other instanceof BoostAttributeImpl) - return ((BoostAttributeImpl) other).boost == boost; + if (other instanceof MaxNonCompetitiveBoostAttributeImpl) { + final MaxNonCompetitiveBoostAttributeImpl o = (MaxNonCompetitiveBoostAttributeImpl) other; + return (o.maxNonCompetitiveBoost == maxNonCompetitiveBoost) + && (o.competitiveTerm == null ? competitiveTerm == null : o.competitiveTerm.equals(competitiveTerm)); + } return false; } @Override public int hashCode() { - return Float.floatToIntBits(boost); + int hash = Float.floatToIntBits(maxNonCompetitiveBoost); + if (competitiveTerm != null) hash = 31 * hash + competitiveTerm.hashCode(); + return hash; } @Override public void copyTo(AttributeImpl target) { - ((BoostAttribute) target).setBoost(boost); + final MaxNonCompetitiveBoostAttributeImpl t = (MaxNonCompetitiveBoostAttributeImpl) target; + t.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost); + t.setCompetitiveTerm(competitiveTerm); } } @@ -177,51 +233,51 @@ private abstract static class BooleanQueryRewrite extends RewriteMethod { protected final int collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException { - final Fields fields = MultiFields.getFields(reader); - if (fields == null) { - // reader has no fields - return 0; - } + final List subReaders = new ArrayList(); + ReaderUtil.gatherSubReaders(subReaders, reader); + int count = 0; + + for (IndexReader r : subReaders) { + final Fields fields = r.fields(); + if (fields == null) { + // reader has no fields + continue; + } - final Terms terms = fields.terms(query.field); - if (terms == null) { - // field does not exist - return 0; - } + final Terms terms = fields.terms(query.field); + if (terms == null) { + // field does not exist + continue; + } - final TermsEnum termsEnum = query.getTermsEnum(reader); - assert termsEnum != null; + final TermsEnum termsEnum = query.getTermsEnum(r, collector.attributes); + assert termsEnum != null; - if (termsEnum == TermsEnum.EMPTY) - return 0; - final BoostAttribute boostAtt = - termsEnum.attributes().addAttribute(BoostAttribute.class); - collector.boostAtt = boostAtt; - int count = 0; - BytesRef bytes; - while ((bytes = termsEnum.next()) != null) { - if (collector.collect(termsEnum, bytes, boostAtt.getBoost())) { - termsEnum.cacheCurrentTerm(); - count++; - } else { - break; + if (termsEnum == TermsEnum.EMPTY) + continue; + collector.setNextEnum(termsEnum); + BytesRef bytes; + while ((bytes = termsEnum.next()) != null) { + if (collector.collect(bytes)) { + termsEnum.cacheCurrentTerm(); + count++; + } else { + return count; // interrupt whole term collection, so also don't iterate other subReaders + } } } - collector.boostAtt = null; return count; } protected static abstract class TermCollector { - private BoostAttribute boostAtt = null; + /** attributes used for communication with the enum */ + public final AttributeSource attributes = new AttributeSource(); /** return false to stop collecting */ - public abstract boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) throws IOException; + public abstract boolean collect(BytesRef bytes) throws IOException; - /** set the minimum boost as a hint for the term producer */ - protected final void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost) { - assert boostAtt != null; - boostAtt.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost); - } + /** the next segment's {@link TermsEnum} that is used to collect terms */ + public abstract void setNextEnum(TermsEnum termsEnum) throws IOException; } } @@ -230,16 +286,50 @@ public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException { final BooleanQuery result = new BooleanQuery(true); final Term placeholderTerm = new Term(query.field); - query.incTotalNumberOfTerms(collectTerms(reader, query, new TermCollector() { + final TermFreqBoostByteStart array = new TermFreqBoostByteStart(16); + final BytesRefHash terms = new BytesRefHash(new ByteBlockPool(new ByteBlockPool.DirectAllocator()), 16, array); + collectTerms(reader, query, new TermCollector() { + private TermsEnum termsEnum; + private BoostAttribute boostAtt; + @Override - public boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) { - // add new TQ, we must clone the term, else it may get overwritten! - TermQuery tq = new TermQuery(placeholderTerm.createTerm(new BytesRef(bytes)), termsEnum.docFreq()); - tq.setBoost(query.getBoost() * boost); // set the boost - result.add(tq, BooleanClause.Occur.SHOULD); // add to query + public void setNextEnum(TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + this.boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class); + } + + @Override + public boolean collect(BytesRef bytes) { + final int e = terms.add(bytes); + if (e < 0 ) { + // duplicate term: update docFreq + final int pos = (-e)-1; + array.docFreq[pos] += termsEnum.docFreq(); + assert array.boost[pos] == boostAtt.getBoost() : "boost should be equal in all segment TermsEnums"; + } else { + // new entry: we populate the entry initially + array.docFreq[e] = termsEnum.docFreq(); + array.boost[e] = boostAtt.getBoost(); + } + // if the new entry reaches the max clause count, we exit early + if (e >= BooleanQuery.getMaxClauseCount()) + throw new BooleanQuery.TooManyClauses(); return true; } - })); + }); + + final int size = terms.size(); + final int[] docFreq = array.docFreq; + final float[] boost = array.boost; + for (int i = 0; i < size; i++) { + final TermQuery tq = new TermQuery( + placeholderTerm.createTerm(terms.get(i, new BytesRef())), + docFreq[i] + ); + tq.setBoost(query.getBoost() * boost[i]); + result.add(tq, BooleanClause.Occur.SHOULD); + } + query.incTotalNumberOfTerms(size); return result; } @@ -247,6 +337,42 @@ protected Object readResolve() { return SCORING_BOOLEAN_QUERY_REWRITE; } + + /** Special implementation of BytesStartArray that keeps parallel arrays for boost and docFreq */ + final static class TermFreqBoostByteStart extends DirectBytesStartArray { + int[] docFreq; + float[] boost; + + public TermFreqBoostByteStart(int initSize) { + super(initSize); + } + + @Override + public int[] init() { + final int[] ord = super.init(); + boost = new float[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_FLOAT)]; + docFreq = new int[ArrayUtil.oversize(ord.length, RamUsageEstimator.NUM_BYTES_INT)]; + assert boost.length >= ord.length && docFreq.length >= ord.length; + return ord; + } + + @Override + public int[] grow() { + final int[] ord = super.grow(); + docFreq = ArrayUtil.grow(docFreq, ord.length); + boost = ArrayUtil.grow(boost, ord.length); + assert boost.length >= ord.length && docFreq.length >= ord.length; + return ord; + } + + @Override + public int[] clear() { + boost = null; + docFreq = null; + return super.clear(); + } + + } } /** A rewrite method that first translates each term into @@ -291,24 +417,64 @@ final int maxSize = Math.min(size, BooleanQuery.getMaxClauseCount()); final PriorityQueue stQueue = new PriorityQueue(); collectTerms(reader, query, new TermCollector() { + private final MaxNonCompetitiveBoostAttribute maxBoostAtt = + attributes.addAttribute(MaxNonCompetitiveBoostAttribute.class); + + private final Map visitedTerms = new HashMap(); + + private TermsEnum termsEnum; + private Comparator termComp; + private BoostAttribute boostAtt; + private ScoreTerm st; + @Override - public boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) { + public void setNextEnum(TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + assert termComp == null || termComp == termsEnum.getComparator() : + "term comparator should not change between segments"; + termComp = termsEnum.getComparator(); + // lazy init the initial ScoreTerm because comparator is not known on ctor: + if (st == null) + st = new ScoreTerm(termComp); + boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class); + } + + @Override + public boolean collect(BytesRef bytes) { + final float boost = boostAtt.getBoost(); // ignore uncompetetive hits - if (stQueue.size() >= maxSize && boost <= stQueue.peek().boost) - return true; + if (stQueue.size() >= maxSize) { + final ScoreTerm t = stQueue.peek(); + if (boost < t.boost) + return true; + if (boost == t.boost && termComp.compare(bytes, t.bytes) > 0) + return true; + } // add new entry in PQ, we must clone the term, else it may get overwritten! st.bytes.copy(bytes); st.boost = boost; st.docFreq = termsEnum.docFreq(); - stQueue.offer(st); - // possibly drop entries from queue - st = (stQueue.size() > maxSize) ? stQueue.poll() : new ScoreTerm(); - setMaxNonCompetitiveBoost((stQueue.size() >= maxSize) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY); + ScoreTerm t = visitedTerms.get(st.bytes); + if (t != null) { + // if the term is already in the queue, only update docFreq + t.docFreq += st.docFreq; + assert t.boost == st.boost : "boost should be equal in all segment TermsEnums"; + } else { + visitedTerms.put(st.bytes, st); + stQueue.offer(st); + // possibly drop entries from queue + if (stQueue.size() > maxSize) { + st = stQueue.poll(); + visitedTerms.remove(st.bytes); + t = stQueue.peek(); + maxBoostAtt.setMaxNonCompetitiveBoost(t.boost); + maxBoostAtt.setCompetitiveTerm(t.bytes); + } else { + st = new ScoreTerm(termComp); + } + } return true; } - - // reusable instance - private ScoreTerm st = new ScoreTerm(); }); final Term placeholderTerm = new Term(query.field); @@ -325,10 +491,7 @@ @Override public int hashCode() { - final int prime = 17; - int result = 1; - result = prime * result + size; - return result; + return 31 * size; } @Override @@ -341,15 +504,20 @@ return true; } - private static class ScoreTerm implements Comparable { + private static final class ScoreTerm implements Comparable { + private final Comparator termComp; + public final BytesRef bytes = new BytesRef(); public float boost; public int docFreq; + public ScoreTerm(Comparator termComp) { + this.termComp = termComp; + } + public int compareTo(ScoreTerm other) { if (this.boost == other.boost) - // TODO: is it OK to use default compare here? - return other.bytes.compareTo(this.bytes); + return termComp.compare(other.bytes, this.bytes); else return Float.compare(this.boost, other.boost); } @@ -510,63 +678,60 @@ final int docCountCutoff = (int) ((docCountPercent / 100.) * reader.maxDoc()); final int termCountLimit = Math.min(BooleanQuery.getMaxClauseCount(), termCountCutoff); - final CutOffTermCollector col = new CutOffTermCollector(reader, query.field, docCountCutoff, termCountLimit); + final CutOffTermCollector col = new CutOffTermCollector(docCountCutoff, termCountLimit); collectTerms(reader, query, col); - + final int size; if (col.hasCutOff) { return CONSTANT_SCORE_FILTER_REWRITE.rewrite(reader, query); - } else if (col.termCount == 0) { + } else if ((size = col.pendingTerms.size()) == 0) { return new BooleanQuery(true); } else { - final PagedBytes.Reader bytesReader = col.pendingTerms.freeze(false); - try { - final BooleanQuery bq = new BooleanQuery(true); - final Term placeholderTerm = new Term(query.field); - long start = col.startOffset; - for(int i = 0; i < col.termCount; i++) { - final BytesRef bytes = new BytesRef(); - start = bytesReader.fillUsingLengthPrefix3(bytes, start); - bq.add(new TermQuery(placeholderTerm.createTerm(bytes)), BooleanClause.Occur.SHOULD); - } - // Strip scores - final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq)); - result.setBoost(query.getBoost()); - query.incTotalNumberOfTerms(col.termCount); - return result; - } finally { - bytesReader.close(); + final BooleanQuery bq = new BooleanQuery(true); + final Term placeholderTerm = new Term(query.field); + final BytesRefHash pendingTerms = col.pendingTerms; + for(int i = 0; i < size; i++) { + // docFreq is not used for constant score here, we pass 1 + // to explicitely set a fake value, so it's not calculated + bq.add(new TermQuery( + placeholderTerm.createTerm(pendingTerms.get(i, new BytesRef())), 1 + ), BooleanClause.Occur.SHOULD); } + // Strip scores + final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq)); + result.setBoost(query.getBoost()); + query.incTotalNumberOfTerms(size); + return result; } } private static final class CutOffTermCollector extends TermCollector { - CutOffTermCollector(IndexReader reader, String field, int docCountCutoff, int termCountLimit) { - this.reader = reader; - this.field = field; + CutOffTermCollector(int docCountCutoff, int termCountLimit) { this.docCountCutoff = docCountCutoff; this.termCountLimit = termCountLimit; } - public boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) throws IOException { - termCount++; - if (termCount >= termCountLimit || docVisitCount >= docCountCutoff) { + @Override + public void setNextEnum(TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + } + + @Override + public boolean collect(BytesRef bytes) throws IOException { + if (pendingTerms.size() >= termCountLimit || docVisitCount >= docCountCutoff) { hasCutOff = true; return false; } - pendingTerms.copyUsingLengthPrefix(bytes); + pendingTerms.add(bytes); docVisitCount += termsEnum.docFreq(); return true; } int docVisitCount = 0; boolean hasCutOff = false; - int termCount = 0; - - final IndexReader reader; - final String field; + TermsEnum termsEnum; + final int docCountCutoff, termCountLimit; - final PagedBytes pendingTerms = new PagedBytes(15); // max term size is 32 KiB - final long startOffset = pendingTerms.getPointer(); + final BytesRefHash pendingTerms = new BytesRefHash(); } @Override @@ -644,9 +809,18 @@ * field does exist). This method should not return null * (should instead return {@link TermsEnum#EMPTY} if no * terms match). The TermsEnum must already be - * positioned to the first matching term. */ - protected abstract TermsEnum getTermsEnum(IndexReader reader) throws IOException; + * positioned to the first matching term. + * TODO: Add description for the {@link AttributeSource} + */ + protected abstract TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException; + /** Convenience method, if no attributes are needed: + * This simply passes emty attributes. + */ + protected final TermsEnum getTermsEnum(IndexReader reader) throws IOException { + return getTermsEnum(reader, new AttributeSource()); + } + /** * Expert: Return the number of unique terms visited during execution of the query. * If there are many of them, you may consider using another query type Index: lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java (working copy) @@ -26,6 +26,7 @@ import org.apache.lucene.util.NumericUtils; import org.apache.lucene.util.ToStringUtils; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; import org.apache.lucene.index.TermsEnum; @@ -301,7 +302,7 @@ } @Override @SuppressWarnings("unchecked") - protected TermsEnum getTermsEnum(final IndexReader reader) throws IOException { + protected TermsEnum getTermsEnum(final IndexReader reader, AttributeSource atts) throws IOException { // very strange: java.lang.Number itsself is not Comparable, but all subclasses used here are return (min != null && max != null && ((Comparable) min).compareTo(max) > 0) ? TermsEnum.EMPTY : Index: lucene/src/java/org/apache/lucene/search/PrefixQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/PrefixQuery.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/PrefixQuery.java (working copy) @@ -24,6 +24,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.Terms; import org.apache.lucene.index.MultiFields; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.ToStringUtils; /** A Query that matches documents containing terms with a specified prefix. A PrefixQuery @@ -45,7 +46,7 @@ public Term getPrefix() { return prefix; } @Override - protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { + protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException { if (prefix.bytes().length == 0) { // no prefix -- match all terms for this field: // NOTE: for now, MultiTermQuery enums terms at the Index: lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java =================================================================== --- lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/QueryWrapperFilter.java (working copy) @@ -33,7 +33,7 @@ * once per day. */ public class QueryWrapperFilter extends Filter { - private Query query; + private final Query query; /** Constructs a filter which only matches documents matching * query. @@ -41,6 +41,11 @@ public QueryWrapperFilter(Query query) { this.query = query; } + + /** returns the inner Query */ + public final Query getQuery() { + return query; + } @Override public DocIdSet getDocIdSet(final IndexReader reader) throws IOException { Index: lucene/src/java/org/apache/lucene/search/TermRangeQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (working copy) @@ -25,6 +25,7 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.ToStringUtils; +import org.apache.lucene.util.AttributeSource; /** * A Query that matches documents within an range of terms. @@ -130,7 +131,7 @@ public Collator getCollator() { return collator; } @Override - protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { + protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException { if (collator == null && lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) { return TermsEnum.EMPTY; } Index: lucene/src/java/org/apache/lucene/util/ArrayUtil.java =================================================================== --- lucene/src/java/org/apache/lucene/util/ArrayUtil.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/util/ArrayUtil.java (working copy) @@ -27,14 +27,7 @@ public final class ArrayUtil { - /** - * @deprecated This constructor was not intended to be public and should not be used. - * This class contains solely a static utility methods. - * It will be made private in Lucene 4.0 - */ - // make private in 4.0! - @Deprecated - public ArrayUtil() {} // no instance + private ArrayUtil() {} // no instance /* Begin Apache Harmony code @@ -247,7 +240,20 @@ public static short[] grow(short[] array) { return grow(array, 1 + array.length); } + + public static float[] grow(float[] array, int minSize) { + if (array.length < minSize) { + float[] newArray = new float[oversize(minSize, RamUsageEstimator.NUM_BYTES_FLOAT)]; + System.arraycopy(array, 0, newArray, 0, array.length); + return newArray; + } else + return array; + } + public static float[] grow(float[] array) { + return grow(array, 1 + array.length); + } + public static short[] shrink(short[] array, int targetSize) { final int newSize = getShrinkSize(array.length, targetSize, RamUsageEstimator.NUM_BYTES_SHORT); if (newSize != array.length) { Index: lucene/src/java/org/apache/lucene/util/ByteBlockPool.java =================================================================== --- lucene/src/java/org/apache/lucene/util/ByteBlockPool.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/util/ByteBlockPool.java (working copy) @@ -16,8 +16,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +import java.util.Arrays; +import java.util.List; +import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF; -/* Class that Posting and PostingVector use to write byte +/** + * Class that Posting and PostingVector use to write byte * streams into shared fixed-size byte[] arrays. The idea * is to allocate slices of increasing lengths For * example, the first slice is 5 bytes, the next slice is @@ -31,14 +35,10 @@ * the end with a non-zero byte. This way the methods * that are writing into the slice don't need to record * its length and instead allocate a new slice once they - * hit a non-zero byte. */ - -import java.util.Arrays; - - -import java.util.List; -import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF; - + * hit a non-zero byte. + * + * @lucene.internal + **/ public final class ByteBlockPool { public final static int BYTE_BLOCK_SHIFT = 15; public final static int BYTE_BLOCK_SIZE = 1 << BYTE_BLOCK_SHIFT; @@ -62,7 +62,23 @@ return new byte[blockSize]; } } + + public static final class DirectAllocator extends Allocator { + + public DirectAllocator() { + this(BYTE_BLOCK_SIZE); + } + public DirectAllocator(int blockSize) { + super(blockSize); + } + + @Override + public void recycleByteBlocks(byte[][] blocks, int start, int end) { + } + + } + public byte[][] buffers = new byte[10][]; int bufferUpto = -1; // Which buffer we are upto Index: lucene/src/java/org/apache/lucene/util/BytesRefHash.java =================================================================== --- lucene/src/java/org/apache/lucene/util/BytesRefHash.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/util/BytesRefHash.java (working copy) @@ -17,13 +17,15 @@ * limitations under the License. */ +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK; +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT; +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE; + import java.util.Arrays; import java.util.Comparator; import java.util.concurrent.atomic.AtomicLong; -import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK; -import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE; -import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SHIFT; +import org.apache.lucene.util.ByteBlockPool.DirectAllocator; /** * {@link BytesRefHash} is a special purpose hash-map like data-structure @@ -54,6 +56,14 @@ public static final int DEFAULT_CAPACITY = 16; private final BytesStartArray bytesStartArray; private AtomicLong bytesUsed; + + /** + * Creates a new {@link BytesRefHash} with a {@link ByteBlockPool} using a + * {@link DirectAllocator}. + */ + public BytesRefHash() { + this(new ByteBlockPool(new DirectAllocator())); + } /** * Creates a new {@link BytesRefHash} @@ -75,7 +85,7 @@ Arrays.fill(ords, -1); this.bytesStartArray = bytesStartArray; bytesStart = bytesStartArray.init(); - bytesUsed = bytesStartArray.bytesUsed(); + bytesUsed = bytesStartArray.bytesUsed() == null? new AtomicLong(0) : bytesStartArray.bytesUsed();; bytesUsed.addAndGet(hashSize * RamUsageEstimator.NUM_BYTES_INT); } @@ -536,13 +546,13 @@ public abstract AtomicLong bytesUsed(); } - static class DirectBytesStartArray extends BytesStartArray { + public static class DirectBytesStartArray extends BytesStartArray { - private final int initSize; + protected final int initSize; private int[] bytesStart; private final AtomicLong bytesUsed = new AtomicLong(0); - DirectBytesStartArray(int initSize) { + public DirectBytesStartArray(int initSize) { this.initSize = initSize; } Index: lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java (revision 0) +++ lucene/src/test/org/apache/lucene/search/TestMultiTermQueryRewrites.java (revision 0) @@ -0,0 +1,99 @@ +package org.apache.lucene.search; + +/** + * 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 org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.analysis.MockTokenizer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.MultiReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.store.Directory; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +public class TestMultiTermQueryRewrites extends BaseTestRangeFilter { + + static Directory dir; + static IndexReader reader, multiReader; + static IndexSearcher searcher, multiSearcher; + + @BeforeClass + public static void beforeClass() throws Exception { + final String[] data = new String[] { "0 1 2 3 4 5 6", "6 5 3 4 8 7", "2 4 5 6", "3 5 6 1" }; + + dir = newDirectory(); + final RandomIndexWriter writer = new RandomIndexWriter(random, dir, new MockAnalyzer(MockTokenizer.WHITESPACE, false)); + + for (int i = 0; i < data.length; i++) { + Document doc = new Document(); + doc.add(newField("data", data[i], Field.Store.NO, Field.Index.ANALYZED)); + writer.addDocument(doc); + } + writer.optimize(); + writer.close(); + + reader = IndexReader.open(dir, true); + searcher = new IndexSearcher(reader); + multiReader = new MultiReader(new IndexReader[] { + IndexReader.open(dir, true), IndexReader.open(dir, true) + }, true); + multiSearcher = new IndexSearcher(multiReader); + } + + @AfterClass + public static void afterClass() throws Exception { + reader.close(); + multiReader.close(); + dir.close(); + reader = multiReader = null; + searcher = multiSearcher = null; + dir = null; + } + + private void check(MultiTermQuery.RewriteMethod method) throws Exception { + final TermRangeQuery tq = new TermRangeQuery("data", "2", "7", true, true); + tq.setRewriteMethod(method); + final Query q1 = searcher.rewrite(tq); + final Query q2 = multiSearcher.rewrite(tq); + if (VERBOSE) { + System.out.println(); + System.out.println("single segment: " + q1); + System.out.println(" multi segment: " + q2); + } + assertEquals("The multi-segment case must produce same rewritten query", q1, q2); + } + + public void testRewritesWithDuplicateTerms() throws Exception { + check(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE); + + check(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE); + + // use a large PQ here to only test duplicate terms and dont mix up when all scores are equal + check(new MultiTermQuery.TopTermsScoringBooleanQueryRewrite(1024)); + check(new MultiTermQuery.TopTermsBoostOnlyBooleanQueryRewrite(1024)); + + final MultiTermQuery.ConstantScoreAutoRewrite rewrite = new MultiTermQuery.ConstantScoreAutoRewrite(); + rewrite.setTermCountCutoff(Integer.MAX_VALUE); + rewrite.setDocCountPercent(Double.POSITIVE_INFINITY); + check(rewrite); + } + +} Property changes on: lucene\src\test\org\apache\lucene\search\TestMultiTermQueryRewrites.java ___________________________________________________________________ Added: svn:keywords + Date Author Id Revision HeadURL Added: svn:eol-style + native Index: lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java (revision 1022463) +++ lucene/src/test/org/apache/lucene/search/TestPrefixRandom.java (working copy) @@ -29,6 +29,7 @@ import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util._TestUtil; @@ -85,7 +86,7 @@ } @Override - protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { + protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException { return new SimplePrefixTermsEnum(reader, field, prefix); } Index: lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java (revision 1022463) +++ lucene/src/test/org/apache/lucene/search/TestRegexpRandom2.java (working copy) @@ -36,6 +36,7 @@ import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.UnicodeUtil; import org.apache.lucene.util._TestUtil; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.AutomatonTestUtil; import org.apache.lucene.util.automaton.CharacterRunAutomaton; @@ -103,7 +104,7 @@ } @Override - protected TermsEnum getTermsEnum(IndexReader reader) throws IOException { + protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException { return new SimpleAutomatonTermsEnum(reader, field); }