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,24 @@ return new BooleanWeight(searcher); } + private static final Comparator tqComp = new Comparator() { + public int compare(BooleanClause one, BooleanClause two) { + Query oneQ = one.getQuery(); + Query twoQ = two.getQuery(); + boolean oneTQ = oneQ instanceof TermQuery; + boolean twoTQ = twoQ instanceof TermQuery; + if (oneTQ && twoTQ) { + return ((TermQuery) oneQ).getTerm().compareTo(((TermQuery) twoQ).getTerm()); + } else if (oneTQ) { + return -1; + } else if (twoTQ) { + 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 +404,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,10 @@ private final MultiTermQuery.BoostAttribute boostAtt = attributes().addAttribute(MultiTermQuery.BoostAttribute.class); - private float bottom = boostAtt.getMaxNonCompetitiveBoost(); + private final MultiTermQuery.MaxNonCompetitiveBoostAttribute maxBoostAtt; + private float bottom; + private final float minSimilarity; private final float scale_factor; @@ -82,7 +85,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 +95,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(); + // 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())]; @@ -202,7 +207,7 @@ BytesRef term = actualEnum.next(); boostAtt.setBoost(actualBoostAtt.getBoost()); - final float bottom = boostAtt.getMaxNonCompetitiveBoost(); + final float bottom = maxBoostAtt.getMaxNonCompetitiveBoost(); if (bottom != this.bottom) { this.bottom = bottom; // clone the term before potentially doing something with it 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 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.FloatUtil; +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,6 +105,43 @@ 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(); + } + + /** Implementation class for {@link MaxNonCompetitiveBoostAttribute}. */ + public static final class MaxNonCompetitiveBoostAttributeImpl extends AttributeImpl implements MaxNonCompetitiveBoostAttribute { + private float maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY; public void setMaxNonCompetitiveBoost(float maxNonCompetitiveBoost) { this.maxNonCompetitiveBoost = maxNonCompetitiveBoost; @@ -118,26 +153,26 @@ @Override public void clear() { - boost = 1.0f; + maxNonCompetitiveBoost = Float.NEGATIVE_INFINITY; } @Override public boolean equals(Object other) { if (this == other) return true; - if (other instanceof BoostAttributeImpl) - return ((BoostAttributeImpl) other).boost == boost; + if (other instanceof MaxNonCompetitiveBoostAttributeImpl) + return ((MaxNonCompetitiveBoostAttributeImpl) other).maxNonCompetitiveBoost == maxNonCompetitiveBoost; return false; } @Override public int hashCode() { - return Float.floatToIntBits(boost); + return Float.floatToIntBits(maxNonCompetitiveBoost); } @Override public void copyTo(AttributeImpl target) { - ((BoostAttribute) target).setBoost(boost); + ((MaxNonCompetitiveBoostAttribute) target).setMaxNonCompetitiveBoost(maxNonCompetitiveBoost); } } @@ -177,51 +212,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; - + 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(TermsEnum termsEnum) throws IOException; + + /** get the attributes passed to getTermsEnum() */ + AttributeSource attributes(); } } @@ -230,16 +265,53 @@ 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() { - @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 + final TermFreqBoostByteStart array = new TermFreqBoostByteStart(16); + final BytesRefHash terms = new BytesRefHash(new ByteBlockPool(new ByteBlockPool.DirectAllocator()), 16, array); + collectTerms(reader, query, new TermCollector() { + private final AttributeSource atts = new AttributeSource(); // unused + private TermsEnum termsEnum; + private BoostAttribute boostAtt; + + public void setNextEnum(TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + this.boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class); + } + + public AttributeSource attributes() { + return atts; + } + + 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 +319,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 +399,65 @@ 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 final AttributeSource atts = new AttributeSource(); + private final MaxNonCompetitiveBoostAttribute maxBoostAtt = atts.addAttribute(MaxNonCompetitiveBoostAttribute.class); + + private TermsEnum termsEnum; + private BoostAttribute boostAtt; + private float boostLie; + + public void setNextEnum(TermsEnum termsEnum) throws IOException { + this.termsEnum = termsEnum; + //boostLie = FloatUtil.nextAfter(maxBoostAtt.getMaxNonCompetitiveBoost(), Double.POSITIVE_INFINITY); + boostAtt = termsEnum.attributes().addAttribute(BoostAttribute.class); + } + + public AttributeSource attributes() { + // this is a hack: attributes() is always called before the TermsEnum is requested from MTQ: + float minBoost = (stQueue.size() >= maxSize) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY; + boostLie = minBoost; + maxBoostAtt.setMaxNonCompetitiveBoost(FloatUtil.nextAfter(minBoost, Double.NEGATIVE_INFINITY)); + // end hack + return atts; + } + + public boolean collect(BytesRef bytes) { + final float boost = boostAtt.getBoost(); // ignore uncompetetive hits - if (stQueue.size() >= maxSize && boost <= stQueue.peek().boost) + if (stQueue.size() >= maxSize && boost != boostLie && boost <= stQueue.peek().boost) 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); + 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 { + visitedTerms.put(st.bytes, st); + stQueue.offer(st); + // possibly drop entries from queue + if (stQueue.size() > maxSize) { + st = stQueue.poll(); + visitedTerms.remove(st.bytes); + } else { + st = new ScoreTerm(); + } + float minBoost = (stQueue.size() >= maxSize) ? stQueue.peek().boost : Float.NEGATIVE_INFINITY; + if (minBoost == boostLie) { + maxBoostAtt.setMaxNonCompetitiveBoost(FloatUtil.nextAfter(minBoost, Float.NEGATIVE_INFINITY)); + } else { + maxBoostAtt.setMaxNonCompetitiveBoost(minBoost); + } + } return true; } // reusable instance private ScoreTerm st = new ScoreTerm(); + private final Map visitedTerms = new HashMap(); }); final Term placeholderTerm = new Term(query.field); @@ -341,7 +490,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; @@ -510,63 +659,63 @@ 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; + 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(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; } + public AttributeSource attributes() { + return atts; + } + 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(); + private final AttributeSource atts = new AttributeSource(); // unused here } @Override @@ -645,7 +794,7 @@ * (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; + protected abstract TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException; /** * Expert: Return the number of unique terms visited during execution of the query. Index: lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java =================================================================== --- lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (revision 1022463) +++ lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (working copy) @@ -26,6 +26,7 @@ import org.apache.lucene.index.DocsEnum; import org.apache.lucene.index.MultiFields; import org.apache.lucene.util.OpenBitSet; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.Bits; /** @@ -118,7 +119,7 @@ return DocIdSet.EMPTY_DOCIDSET; } - final TermsEnum termsEnum = query.getTermsEnum(reader); + final TermsEnum termsEnum = query.getTermsEnum(reader, new AttributeSource()); assert termsEnum != null; if (termsEnum.next() != null) { // fill into a OpenBitSet 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/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/java/org/apache/lucene/util/FloatUtil.java =================================================================== --- lucene/src/java/org/apache/lucene/util/FloatUtil.java (revision 0) +++ lucene/src/java/org/apache/lucene/util/FloatUtil.java (revision 0) @@ -0,0 +1,75 @@ +package org.apache.lucene.util; + +/** + * 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. + */ + +public class FloatUtil { + /** + * Answers a float next to the first given float value in the direction of + * the second given double value. + * + * @param start + * the float value to start + * @param direction + * the double indicating the direction + * @return a float next to the first given float value in the direction of + * the second given double. + * + * @since 1.6 + */ + @SuppressWarnings("boxing") + public static float nextAfter(float start, double direction) { + /* this implementation is from apache harmony's java 6 branch (StrictMath) */ + if (Float.isNaN(start) || Double.isNaN(direction)) { + return Float.NaN; + } + if (0 == start && 0 == direction) { + return (float)direction; + } + if ((start == Float.MIN_VALUE && direction < start) + || (start == -Float.MIN_VALUE && direction > start)) { + return (start > 0 ? 0f : -0f); + } + if (Float.isInfinite(start) && (direction != start)) { + return (start > 0 ? Float.MAX_VALUE : -Float.MAX_VALUE); + } + if ((start == Float.MAX_VALUE && direction > start) + || (start == -Float.MAX_VALUE && direction < start)) { + return (start > 0 ? Float.POSITIVE_INFINITY + : Float.NEGATIVE_INFINITY); + } + if (direction > start) { + if (start > 0) { + return Float.intBitsToFloat(Float.floatToIntBits(start) + 1); + } + if (start < 0) { + return Float.intBitsToFloat(Float.floatToIntBits(start) - 1); + } + return +Float.MIN_VALUE; + } + if (direction < start) { + if (start > 0) { + return Float.intBitsToFloat(Float.floatToIntBits(start) - 1); + } + if (start < 0) { + return Float.intBitsToFloat(Float.floatToIntBits(start) + 1); + } + return -Float.MIN_VALUE; + } + return (float)direction; + } +} Property changes on: lucene\src\java\org\apache\lucene\util\FloatUtil.java ___________________________________________________________________ Added: svn:keywords + Date Author Id Revision HeadURL Added: svn:eol-style + native Index: lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java (revision 1022463) +++ lucene/src/test/org/apache/lucene/search/TestAutomatonQuery.java (working copy) @@ -26,6 +26,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.BasicAutomata; @@ -170,7 +171,7 @@ public void testRewriteSingleTerm() throws IOException { AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), BasicAutomata .makeString("piece")); - assertTrue(aq.getTermsEnum(searcher.getIndexReader()) instanceof SingleTermsEnum); + assertTrue(aq.getTermsEnum(searcher.getIndexReader(), new AttributeSource()) instanceof SingleTermsEnum); assertEquals(1, automatonQueryNrHits(aq)); } @@ -184,7 +185,7 @@ Automaton prefixAutomaton = BasicOperations.concatenate(pfx, BasicAutomata .makeAnyString()); AutomatonQuery aq = new AutomatonQuery(newTerm("bogus"), prefixAutomaton); - assertTrue(aq.getTermsEnum(searcher.getIndexReader()) instanceof PrefixTermsEnum); + assertTrue(aq.getTermsEnum(searcher.getIndexReader(), new AttributeSource()) instanceof PrefixTermsEnum); assertEquals(3, automatonQueryNrHits(aq)); } @@ -196,7 +197,7 @@ .makeEmpty()); // not yet available: assertTrue(aq.getEnum(searcher.getIndexReader()) // instanceof EmptyTermEnum); - assertSame(TermsEnum.EMPTY, aq.getTermsEnum(searcher.getIndexReader())); + assertSame(TermsEnum.EMPTY, aq.getTermsEnum(searcher.getIndexReader(), new AttributeSource())); assertEquals(0, automatonQueryNrHits(aq)); } } Index: lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java (revision 1022463) +++ lucene/src/test/org/apache/lucene/search/TestNumericRangeQuery32.java (working copy) @@ -26,6 +26,7 @@ import org.apache.lucene.index.RandomIndexWriter; import org.apache.lucene.index.TermsEnum; 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.NumericUtils; @@ -563,7 +564,7 @@ private void testEnum(int lower, int upper) throws Exception { NumericRangeQuery q = NumericRangeQuery.newIntRange("field4", 4, lower, upper, true, true); - TermsEnum termEnum = q.getTermsEnum(searcher.getIndexReader()); + TermsEnum termEnum = q.getTermsEnum(searcher.getIndexReader(), new AttributeSource()); int count = 0; while (termEnum.next() != null) { final BytesRef t = termEnum.term(); Index: lucene/src/test/org/apache/lucene/search/TestPrefixQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestPrefixQuery.java (revision 1022463) +++ lucene/src/test/org/apache/lucene/search/TestPrefixQuery.java (working copy) @@ -24,6 +24,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; +import org.apache.lucene.util.AttributeSource; /** * Tests {@link PrefixQuery} class. @@ -54,7 +55,7 @@ assertEquals("One in /Computers/Mac", 1, hits.length); query = new PrefixQuery(new Term("category", "")); - assertFalse(query.getTermsEnum(searcher.getIndexReader()) instanceof PrefixTermsEnum); + assertFalse(query.getTermsEnum(searcher.getIndexReader(), new AttributeSource()) instanceof PrefixTermsEnum); hits = searcher.search(query, null, 1000).scoreDocs; assertEquals("everything", 3, hits.length); writer.close(); 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); } @@ -154,7 +155,7 @@ // a\uda07* prefixquery. Prefixquery then does the "wrong" thing, which // isn't really wrong as the query was undefined to begin with... but not // automatically comparable. - if (!(smart.getTermsEnum(searcher.getIndexReader()) instanceof AutomatonTermsEnum)) + if (!(smart.getTermsEnum(searcher.getIndexReader(), new AttributeSource()) instanceof AutomatonTermsEnum)) return; TopDocs smartDocs = searcher.search(smart, 25); Index: lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java (revision 1022463) +++ lucene/src/test/org/apache/lucene/search/TestTermRangeQuery.java (working copy) @@ -28,6 +28,7 @@ import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.LuceneTestCase; import java.io.IOException; @@ -103,17 +104,19 @@ initializeIndex(new String[]{"A", "B", "C", "D"}); IndexSearcher searcher = new IndexSearcher(dir, true); TermRangeQuery query = new TermRangeQuery("content", null, null, true, true); - assertFalse(query.getTermsEnum(searcher.getIndexReader()) instanceof TermRangeTermsEnum); + + final AttributeSource atts = new AttributeSource(); // dummy + assertFalse(query.getTermsEnum(searcher.getIndexReader(), atts) instanceof TermRangeTermsEnum); assertEquals(4, searcher.search(query, null, 1000).scoreDocs.length); query = new TermRangeQuery("content", null, null, false, false); - assertFalse(query.getTermsEnum(searcher.getIndexReader()) instanceof TermRangeTermsEnum); + assertFalse(query.getTermsEnum(searcher.getIndexReader(), atts) instanceof TermRangeTermsEnum); assertEquals(4, searcher.search(query, null, 1000).scoreDocs.length); query = new TermRangeQuery("content", "", null, true, false); - assertFalse(query.getTermsEnum(searcher.getIndexReader()) instanceof TermRangeTermsEnum); + assertFalse(query.getTermsEnum(searcher.getIndexReader(), atts) instanceof TermRangeTermsEnum); assertEquals(4, searcher.search(query, null, 1000).scoreDocs.length); // and now anothe one query = new TermRangeQuery("content", "B", null, true, false); - assertTrue(query.getTermsEnum(searcher.getIndexReader()) instanceof TermRangeTermsEnum); + assertTrue(query.getTermsEnum(searcher.getIndexReader(), atts) instanceof TermRangeTermsEnum); assertEquals(3, searcher.search(query, null, 1000).scoreDocs.length); searcher.close(); } Index: lucene/src/test/org/apache/lucene/search/TestWildcard.java =================================================================== --- lucene/src/test/org/apache/lucene/search/TestWildcard.java (revision 1022463) +++ lucene/src/test/org/apache/lucene/search/TestWildcard.java (working copy) @@ -18,6 +18,7 @@ */ import org.apache.lucene.store.Directory; +import org.apache.lucene.util.AttributeSource; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.document.Document; @@ -129,12 +130,13 @@ MultiTermQuery wq = new WildcardQuery(new Term("field", "prefix*")); assertMatches(searcher, wq, 2); - assertTrue(wq.getTermsEnum(searcher.getIndexReader()) instanceof PrefixTermsEnum); + final AttributeSource atts = new AttributeSource(); // dummy + assertTrue(wq.getTermsEnum(searcher.getIndexReader(), atts) instanceof PrefixTermsEnum); wq = new WildcardQuery(new Term("field", "*")); assertMatches(searcher, wq, 2); - assertFalse(wq.getTermsEnum(searcher.getIndexReader()) instanceof PrefixTermsEnum); - assertFalse(wq.getTermsEnum(searcher.getIndexReader()) instanceof AutomatonTermsEnum); + assertFalse(wq.getTermsEnum(searcher.getIndexReader(), atts) instanceof PrefixTermsEnum); + assertFalse(wq.getTermsEnum(searcher.getIndexReader(), atts) instanceof AutomatonTermsEnum); searcher.close(); indexStore.close(); }