diff -r 3e170a88fba8 lucene/src/java/org/apache/lucene/search/BooleanQuery.java --- a/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Sat Oct 09 06:28:33 2010 -0400 +++ b/lucene/src/java/org/apache/lucene/search/BooleanQuery.java Sun Oct 10 07:10:57 2010 -0400 @@ -31,7 +31,7 @@ */ public class BooleanQuery extends Query implements Iterable { - private static int maxClauseCount = 1024; + private static int maxClauseCount = 102400; /** Thrown when an attempt is made to add more than {@link * #getMaxClauseCount()} clauses. This typically happens if @@ -322,7 +322,7 @@ optional.add(subScorer); } } - + // Check if we can return a BooleanScorer if (!scoreDocsInOrder && topScorer && required.size() == 0 && prohibited.size() < 32) { return new BooleanScorer(this, similarity, minNrShouldMatch, optional, prohibited, maxCoord); diff -r 3e170a88fba8 lucene/src/java/org/apache/lucene/search/MultiTermQuery.java --- a/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Sat Oct 09 06:28:33 2010 -0400 +++ b/lucene/src/java/org/apache/lucene/search/MultiTermQuery.java Sun Oct 10 07:10:57 2010 -0400 @@ -20,6 +20,11 @@ import java.io.IOException; import java.io.Serializable; import java.util.PriorityQueue; +import java.util.HashMap; +import java.util.TreeMap; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.Term; @@ -31,7 +36,11 @@ import org.apache.lucene.queryParser.QueryParser; // for javadoc import org.apache.lucene.util.Attribute; import org.apache.lucene.util.AttributeImpl; -import org.apache.lucene.util.PagedBytes; +import org.apache.lucene.util.BytesRefHash; +import org.apache.lucene.util.ByteBlockPool; +import org.apache.lucene.util.RecyclingByteBlockAllocator; +import org.apache.lucene.util.ReaderUtil; +import org.apache.lucene.util.Bits; /** * An abstract {@link Query} that matches documents @@ -177,51 +186,48 @@ 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); + 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(r, 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; - + protected static interface TermCollector { /** return false to stop collecting */ - public abstract boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) throws IOException; + 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 */ + void setNextEnum(IndexReader reader, TermsEnum termsEnum) throws IOException; } } @@ -230,23 +236,84 @@ public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException { final BooleanQuery result = new BooleanQuery(true); final Term placeholderTerm = new Term(query.field); + final Map terms = new TreeMap(); + final Map> subScorers = new HashMap>(); query.incTotalNumberOfTerms(collectTerms(reader, query, new TermCollector() { - @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 + private TermsEnum termsEnum; + private BoostAttribute boostAtt; + private Map segSubScorers; + private Bits delDocs; + private byte[] norms; + + public void setNextEnum(IndexReader reader, TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + this.delDocs = reader.getDeletedDocs(); + this.boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class); + segSubScorers = new TreeMap(); + subScorers.put(reader, segSubScorers); + norms = reader.norms(query.getField()); + } + + public boolean collect(BytesRef bytes) throws IOException { + TermFreqBoost tfb = terms.get(bytes); + if (tfb != null) { + tfb.docFreq += termsEnum.docFreq(); + assert tfb.boost == boostAtt.getBoost() : "boost should be equal in all segment TermsEnums"; + } else { + // we must clone the term, else it may get + // overwritten! + tfb = new TermFreqBoost(termsEnum.docFreq(), boostAtt.getBoost(), + new TermQuery.TermWeight(placeholderTerm.createTerm(bytes), query.getBoost()*boostAtt.getBoost())); + terms.put(new BytesRef(bytes), tfb); + assert segSubScorers.get(bytes) == null; + } + + if (segSubScorers.get(bytes) == null) { + // nocommit -- can we save this clone by somehow + // getting the already-cloned key from terms + // map? + segSubScorers.put(new BytesRef(bytes), + new TermScorer(tfb.weight, + termsEnum.docs(delDocs, null), + Similarity.getDefault(), // nocommit -- must somehow get from searcher, + norms)); + } + //System.out.println(" coll size=" + terms.size()); + if (terms.size() > BooleanQuery.getMaxClauseCount()) + throw new BooleanQuery.TooManyClauses(); return true; } })); - return result; + + // System.out.println("DEDUP count " + terms.size()); + float sum = 0.0f; + Searcher is = new IndexSearcher(reader); + for (Map.Entry e : terms.entrySet()) { + final TermQuery.TermWeight w = e.getValue().weight; + w.recompute(e.getValue().docFreq, is); + sum += w.sumOfSquaredWeights(); + } + sum *= query.getBoost() * query.getBoost(); + //System.out.println(" REWRITE DONE"); + return new FakeQuery(subScorers, sum); } // Make sure we are still a singleton even after deserializing protected Object readResolve() { return SCORING_BOOLEAN_QUERY_REWRITE; } + + private final class TermFreqBoost { + TermFreqBoost(int docFreq, float boost, TermQuery.TermWeight weight) { + this.docFreq = docFreq; + this.boost = boost; + this.weight = weight; + } + + int docFreq; + float boost; + TermQuery.TermWeight weight; + } } /** A rewrite method that first translates each term into @@ -291,8 +358,17 @@ final int maxSize = Math.min(size, BooleanQuery.getMaxClauseCount()); final PriorityQueue stQueue = new PriorityQueue(); collectTerms(reader, query, new TermCollector() { - @Override - public boolean collect(TermsEnum termsEnum, BytesRef bytes, float boost) { + private TermsEnum termsEnum; + private BoostAttribute boostAtt; + + public void setNextEnum(IndexReader reader, TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class); + boostAtt.setMaxNonCompetitiveBoost((stQueue.size() >= maxSize) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY); + } + + public boolean collect(BytesRef bytes) { + final float boost = boostAtt.getBoost(); // ignore uncompetetive hits if (stQueue.size() >= maxSize && boost <= stQueue.peek().boost) return true; @@ -300,15 +376,28 @@ 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); + final ScoreTerm st2 = visitedTerms.get(st.bytes); + if (st2 != null) { + // if the term is already in the queue, only update docFreq + st2.docFreq += st.docFreq; + assert st2.boost == st.boost : "boost should be equal in all segment TermsEnums"; + } else { + stQueue.offer(st); + // possibly drop entries from queue + if (stQueue.size() > maxSize) { + st = stQueue.poll(); + visitedTerms.remove(st.bytes); + } else { + st = new ScoreTerm(); + } + boostAtt.setMaxNonCompetitiveBoost((stQueue.size() >= maxSize) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY); + } return true; } // reusable instance private ScoreTerm st = new ScoreTerm(); + private final Map visitedTerms = new HashMap(); }); final Term placeholderTerm = new Term(query.field); @@ -341,7 +430,7 @@ return true; } - private static class ScoreTerm implements Comparable { + private static final class ScoreTerm implements Comparable { public final BytesRef bytes = new BytesRef(); public float boost; public int docFreq; @@ -422,10 +511,14 @@ @Override public Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException { Query result = super.rewrite(reader, query); - assert result instanceof BooleanQuery; - // TODO: if empty boolean query return NullQuery? - if (!((BooleanQuery) result).clauses().isEmpty()) { - // strip the scores off + if (result instanceof BooleanQuery) { + // TODO: if empty boolean query return NullQuery? + if (!((BooleanQuery) result).clauses().isEmpty()) { + // strip the scores off + result = new ConstantScoreQuery(new QueryWrapperFilter(result)); + result.setBoost(query.getBoost()); + } + } else if (result instanceof FakeQuery) { result = new ConstantScoreQuery(new QueryWrapperFilter(result)); result.setBoost(query.getBoost()); } @@ -510,63 +603,56 @@ 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); if (col.hasCutOff) { return CONSTANT_SCORE_FILTER_REWRITE.rewrite(reader, query); - } else if (col.termCount == 0) { + } else if (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); + for(int i = 0, c = col.pendingTerms.size(); i < c; i++) { + final BytesRef bytes = new BytesRef(col.pendingTerms.get(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(bytes), 1), BooleanClause.Occur.SHOULD); } + // Strip scores + final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq)); + result.setBoost(query.getBoost()); + query.incTotalNumberOfTerms(col.pendingTerms.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; + private static final class CutOffTermCollector implements TermCollector { + 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) { + public void setNextEnum(IndexReader reader, TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + } + + 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(new ByteBlockPool(new RecyclingByteBlockAllocator())); } @Override @@ -727,3 +813,72 @@ } } + +class FakeQuery extends Query { + private final Map> subScorers; + private final float sumSqWeights; + + public FakeQuery(Map> subScorers, float sumSqWeights) { + this.subScorers = subScorers; + this.sumSqWeights = sumSqWeights; + } + + class FakeWeight extends Weight { + private final float norm; + + public FakeWeight(Searcher searcher) { + norm = getSimilarity(searcher).queryNorm(sumSqWeights); + } + + @Override + public String toString() { return "weight(" + null + ")"; } + + @Override + public Query getQuery() { return null; } + + @Override + public float getValue() { return 1.0f; } + + @Override + public float sumOfSquaredWeights() { + return 1.0f; + } + + @Override + public void normalize(float queryNorm) { + } + + @Override + public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException { + List subs = new ArrayList(); + for(TermScorer ts : subScorers.get(reader).values()) { + ts.resetWeight(norm); + subs.add(ts); + } + if (subs.size() == 0) { + return null; + } else if (subs.size() == 1) { + return subs.get(0); + } else { + return new DisjunctionSumScorer(subs, 1); + } + } + + @Override + public Explanation explain(IndexReader reader, int doc) + throws IOException { + // nocommit TODO + return null; + } + } + + @Override + public String toString(String field) { + return "FakeQuery"; + } + + @Override + public Weight createWeight(Searcher searcher) { + return new FakeWeight(searcher); + } +} \ No newline at end of file diff -r 3e170a88fba8 lucene/src/java/org/apache/lucene/search/TermQuery.java --- a/lucene/src/java/org/apache/lucene/search/TermQuery.java Sat Oct 09 06:28:33 2010 -0400 +++ b/lucene/src/java/org/apache/lucene/search/TermQuery.java Sun Oct 10 07:10:57 2010 -0400 @@ -33,17 +33,28 @@ private final Term term; private final int docFreq; - private class TermWeight extends Weight { - private final Similarity similarity; + static class TermWeight extends Weight { + private Similarity similarity; private float value; private float idf; private float queryNorm; private float queryWeight; private IDFExplanation idfExp; + private Term term; + private float boost; + private TermQuery query; - public TermWeight(Searcher searcher) + public TermWeight(Term t, float boost) { + this.term = t; + this.boost = boost; + } + + public TermWeight(Searcher searcher, Term t, int docFreq, float boost, TermQuery query) throws IOException { - this.similarity = getSimilarity(searcher); + this.boost = boost; + this.similarity = searcher.getSimilarity(); + this.term = t; + this.query = query; if (docFreq != -1) { idfExp = similarity.idfExplain(term, searcher, docFreq); } else { @@ -52,18 +63,23 @@ idf = idfExp.getIdf(); } - @Override - public String toString() { return "weight(" + TermQuery.this + ")"; } + public void recompute(int docFreq, Searcher searcher) throws IOException { + idfExp = searcher.getSimilarity().idfExplain(term, searcher, docFreq); + idf = idfExp.getIdf(); + } @Override - public Query getQuery() { return TermQuery.this; } + public String toString() { return "weight(" + query + ")"; } + + @Override + public Query getQuery() { return query; } @Override public float getValue() { return value; } @Override public float sumOfSquaredWeights() { - queryWeight = idf * getBoost(); // compute query weight + queryWeight = idf * boost; return queryWeight * queryWeight; // square it } @@ -100,8 +116,8 @@ Explanation queryExpl = new Explanation(); queryExpl.setDescription("queryWeight(" + getQuery() + "), product of:"); - Explanation boostExpl = new Explanation(getBoost(), "boost"); - if (getBoost() != 1.0f) + Explanation boostExpl = new Explanation(boost, "boost"); + if (boost != 1.0f) queryExpl.addDetail(boostExpl); queryExpl.addDetail(expl); @@ -181,7 +197,7 @@ @Override public Weight createWeight(Searcher searcher) throws IOException { - return new TermWeight(searcher); + return new TermWeight(searcher, term, docFreq, getBoost(), this); } @Override diff -r 3e170a88fba8 lucene/src/java/org/apache/lucene/search/TermScorer.java --- a/lucene/src/java/org/apache/lucene/search/TermScorer.java Sat Oct 09 06:28:33 2010 -0400 +++ b/lucene/src/java/org/apache/lucene/search/TermScorer.java Sun Oct 10 07:10:57 2010 -0400 @@ -58,9 +58,18 @@ this.docsEnum = td; this.norms = norms; - this.weightValue = weight.getValue(); bulkResult = td.getBulkResult(); + weightValue = weight.getValue(); + setScoreCache(); + } + public void resetWeight(float norm) { + weight.normalize(norm); + weightValue = weight.getValue(); + // xoSystem.out.println("TS=" + weight + " weightValue=" + weightValue); + } + + public void setScoreCache() { for (int i = 0; i < SCORE_CACHE_SIZE; i++) scoreCache[i] = getSimilarity().tf(i) * weightValue; } diff -r 3e170a88fba8 lucene/src/test/org/apache/lucene/TestDemo.java --- a/lucene/src/test/org/apache/lucene/TestDemo.java Sat Oct 09 06:28:33 2010 -0400 +++ b/lucene/src/test/org/apache/lucene/TestDemo.java Sun Oct 10 07:10:57 2010 -0400 @@ -31,6 +31,8 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; +import org.apache.lucene.search.MultiTermQuery; +import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.store.Directory; import org.apache.lucene.util.LuceneTestCase; @@ -78,6 +80,12 @@ query = parser.parse("\"to be\""); assertEquals(1, isearcher.search(query, null, 1).totalHits); + // Test simple prefix query + System.out.println("TEST: now prefix"); + query = parser.parse("t*"); + ((PrefixQuery) query).setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE); + assertEquals(1, isearcher.search(query, null, 1).totalHits); + isearcher.close(); directory.close(); }