Index: lucene/src/java/org/apache/lucene/search/MultiTermQuery.java =================================================================== --- lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (revision 957725) +++ lucene/src/java/org/apache/lucene/search/MultiTermQuery.java (working copy) @@ -32,6 +32,7 @@ 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; /** * An abstract {@link Query} that matches documents @@ -177,11 +178,6 @@ private abstract static class BooleanQueryRewrite extends RewriteMethod { protected final int collectTerms(IndexReader reader, MultiTermQuery query, TermCollector collector) throws IOException { - - if (query.field == null) { - throw new NullPointerException("If you implement getTermsEnum(), you must specify a non-null field in the constructor of MultiTermQuery."); - } - final Fields fields = MultiFields.getFields(reader); if (fields == null) { // reader has no fields @@ -203,10 +199,9 @@ termsEnum.attributes().addAttribute(BoostAttribute.class); collector.boostAtt = boostAtt; int count = 0; - BytesRef term; - final Term placeholderTerm = new Term(query.field); - while ((term = termsEnum.next()) != null) { - if (collector.collect(placeholderTerm.createTerm(term.utf8ToString()), boostAtt.getBoost())) { + BytesRef bytes; + while ((bytes = termsEnum.next()) != null) { + if (collector.collect(bytes, boostAtt.getBoost())) { count++; } else { break; @@ -217,15 +212,15 @@ } protected static abstract class TermCollector { - /** this field is only set if a boostAttribute is used (e.g. {@link FuzzyTermsEnum}) */ private BoostAttribute boostAtt = null; /** return false to stop collecting */ - public abstract boolean collect(Term t, float boost) throws IOException; + public abstract boolean collect(BytesRef bytes, float boost) throws IOException; /** set the minimum boost as a hint for the term producer */ protected final void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost) { - if (boostAtt != null) boostAtt.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost); + assert boostAtt != null; + boostAtt.setMaxNonCompetitiveBoost(maxNonCompetitiveBoost); } } } @@ -234,9 +229,11 @@ @Override 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() { - public boolean collect(Term t, float boost) { - TermQuery tq = new TermQuery(t); // found a match + public boolean collect(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))); tq.setBoost(query.getBoost() * boost); // set the boost result.add(tq, BooleanClause.Occur.SHOULD); // add to query return true; @@ -297,16 +294,16 @@ protected abstract Query getQuery(Term term); @Override - public Query rewrite(IndexReader reader, MultiTermQuery query) throws IOException { + public Query rewrite(final IndexReader reader, final MultiTermQuery query) throws IOException { final int maxSize = Math.min(size, BooleanQuery.getMaxClauseCount()); final PriorityQueue stQueue = new PriorityQueue(); collectTerms(reader, query, new TermCollector() { - public boolean collect(Term t, float boost) { + public boolean collect(BytesRef bytes, float boost) { // ignore uncompetetive hits if (stQueue.size() >= maxSize && boost <= stQueue.peek().boost) return true; - // add new entry in PQ - st.term = t; + // add new entry in PQ, we must clone the term, else it may get overwritten! + st.bytes.copy(bytes); st.boost = boost; stQueue.offer(st); // possibly drop entries from queue @@ -319,9 +316,11 @@ private ScoreTerm st = new ScoreTerm(); }); + final Term placeholderTerm = new Term(query.field); final BooleanQuery bq = new BooleanQuery(true); for (final ScoreTerm st : stQueue) { - Query tq = getQuery(st.term); // found a match + // add new query, we must clone the term, else it may get overwritten! + Query tq = getQuery(placeholderTerm.createTerm(st.bytes)); tq.setBoost(query.getBoost() * st.boost); // set the boost bq.add(tq, BooleanClause.Occur.SHOULD); // add to query } @@ -348,12 +347,13 @@ } private static class ScoreTerm implements Comparable { - public Term term; + public final BytesRef bytes = new BytesRef(); public float boost; public int compareTo(ScoreTerm other) { if (this.boost == other.boost) - return other.term.compareTo(this.term); + // TODO: is it OK to use default compare here? + return other.bytes.compareTo(this.bytes); else return Float.compare(this.boost, other.boost); } @@ -530,58 +530,67 @@ final int docCountCutoff = (int) ((docCountPercent / 100.) * reader.maxDoc()); final int termCountLimit = Math.min(BooleanQuery.getMaxClauseCount(), termCountCutoff); - final CutOffTermCollector col = new CutOffTermCollector(reader, docCountCutoff, termCountLimit); + final CutOffTermCollector col = new CutOffTermCollector(reader, query.field, docCountCutoff, termCountLimit); collectTerms(reader, query, col); if (col.hasCutOff) { return CONSTANT_SCORE_FILTER_REWRITE.rewrite(reader, query); + } else if (col.termCount == 0) { + return new BooleanQuery(true); } else { - final Query result; - if (col.pendingTerms.isEmpty()) { - result = new BooleanQuery(true); - } else { - BooleanQuery bq = new BooleanQuery(true); - for(Term term : col.pendingTerms) { - TermQuery tq = new TermQuery(term); - bq.add(tq, BooleanClause.Occur.SHOULD); + 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 - result = new ConstantScoreQuery(new QueryWrapperFilter(bq)); + final Query result = new ConstantScoreQuery(new QueryWrapperFilter(bq)); result.setBoost(query.getBoost()); + query.incTotalNumberOfTerms(col.termCount); + return result; + } finally { + bytesReader.close(); } - query.incTotalNumberOfTerms(col.pendingTerms.size()); - return result; } } private static final class CutOffTermCollector extends TermCollector { - CutOffTermCollector(IndexReader reader, int docCountCutoff, int termCountLimit) { + CutOffTermCollector(IndexReader reader, String field, int docCountCutoff, int termCountLimit) { this.reader = reader; + this.field = field; this.docCountCutoff = docCountCutoff; this.termCountLimit = termCountLimit; } - public boolean collect(Term t, float boost) throws IOException { - pendingTerms.add(t); - if (pendingTerms.size() >= termCountLimit || docVisitCount >= docCountCutoff) { + public boolean collect(BytesRef bytes, float boost) throws IOException { + termCount++; + if (termCount >= termCountLimit || docVisitCount >= docCountCutoff) { hasCutOff = true; return false; } + pendingTerms.copyUsingLengthPrefix(bytes); // Loading the TermInfo from the terms dict here // should not be costly, because 1) the // query/filter will load the TermInfo when it // runs, and 2) the terms dict has a cache: - // @deprecated: in 4.0 use BytesRef for collectTerms() - docVisitCount += reader.docFreq(t); + docVisitCount += reader.docFreq(field, bytes); return true; } int docVisitCount = 0; boolean hasCutOff = false; + int termCount = 0; final IndexReader reader; + final String field; final int docCountCutoff, termCountLimit; - final ArrayList pendingTerms = new ArrayList(); + final PagedBytes pendingTerms = new PagedBytes(15); // max term size is 32 KiB + final long startOffset = pendingTerms.getPointer(); } @Override @@ -647,20 +656,9 @@ */ public MultiTermQuery(final String field) { this.field = field; + assert field != null; } - /** - * Constructs a query matching terms that cannot be represented with a single - * Term. - * @deprecated Use {@link #MultiTermQuery(String)}, as the flex branch can - * only work on one field per terms enum. If you override - * {@link #getTermsEnum(IndexReader)}, you cannot use this ctor. - */ - @Deprecated - public MultiTermQuery() { - this(null); - } - /** Returns the field name for this query */ public final String getField() { return field; } Index: lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java =================================================================== --- lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (revision 957725) +++ lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (working copy) @@ -106,10 +106,6 @@ */ @Override public DocIdSet getDocIdSet(IndexReader reader) throws IOException { - if (query.field == null) { - throw new NullPointerException("If you implement getTermsEnum(), you must specify a non-null field in the constructor of MultiTermQuery."); - } - final Fields fields = MultiFields.getFields(reader); if (fields == null) { // reader has no fields Index: lucene/src/java/org/apache/lucene/util/PagedBytes.java =================================================================== --- lucene/src/java/org/apache/lucene/util/PagedBytes.java (revision 957725) +++ lucene/src/java/org/apache/lucene/util/PagedBytes.java (working copy) @@ -125,7 +125,26 @@ return index; } + /** @lucene.internal Reads length as 1 or 2 byte vInt prefix, starting @ start. + * Returns the start offset of the next part, suitable for start parameter on next call. */ + public long fillUsingLengthPrefix3(BytesRef b, long start) { + final int index = (int) (start >> blockBits); + final int offset = (int) (start & blockMask); + final byte[] block = b.bytes = blocks[index]; + if ((block[offset] & 128) == 0) { + b.length = block[offset]; + b.offset = offset+1; + start += 1L + b.length; + } else { + b.length = (((int) (block[offset] & 0x7f)) << 8) | (block[1+offset] & 0xff); + b.offset = offset+2; + start += 2L + b.length; + assert b.length > 0; + } + return start; + } + /** @lucene.internal */ public byte[][] getBlocks() { return blocks;