You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by us...@apache.org on 2009/04/16 14:58:29 UTC

svn commit: r765618 [1/2] - in /lucene/java/trunk/contrib/queries/src: java/org/apache/lucene/search/trie/ test/org/apache/lucene/search/trie/

Author: uschindler
Date: Thu Apr 16 12:58:28 2009
New Revision: 765618

URL: http://svn.apache.org/viewvc?rev=765618&view=rev
Log:
LUCENE-1602: Rewrite TrieRange to use MultiTermQuery

Added:
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeQuery.java   (with props)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeQuery.java   (with props)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeQuery.java   (with props)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeTermEnum.java   (with props)
    lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestIntTrieRangeQuery.java   (with props)
    lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestLongTrieRangeQuery.java   (with props)
Removed:
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeFilter.java
    lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestIntTrieRangeFilter.java
    lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestLongTrieRangeFilter.java
Modified:
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java   (contents, props changed)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieTokenStream.java   (contents, props changed)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java   (contents, props changed)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieTokenStream.java   (contents, props changed)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/ShiftAttribute.java   (contents, props changed)
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/package.html
    lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestIntTrieTokenStream.java   (contents, props changed)
    lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestLongTrieTokenStream.java   (contents, props changed)

Added: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeQuery.java?rev=765618&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeQuery.java (added)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeQuery.java Thu Apr 16 12:58:28 2009
@@ -0,0 +1,112 @@
+package org.apache.lucene.search.trie;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.FilteredTermEnum;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.ToStringUtils;
+
+abstract class AbstractTrieRangeQuery extends MultiTermQuery {
+
+  AbstractTrieRangeQuery(final String field, final int precisionStep,
+    Number min, Number max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    this.field = field.intern();
+    this.precisionStep = precisionStep;
+    this.min = min;
+    this.max = max;
+    this.minInclusive = minInclusive;
+    this.maxInclusive = maxInclusive;
+    setConstantScoreRewrite(true);
+  }
+  
+  abstract void passRanges(TrieRangeTermEnum enumerator);
+
+  //@Override
+  protected FilteredTermEnum getEnum(final IndexReader reader) throws IOException {
+    TrieRangeTermEnum enumerator = new TrieRangeTermEnum(this, reader);
+    passRanges(enumerator);
+    enumerator.init();
+    return enumerator;
+  }
+
+  /** Returns the field name for this query */
+  public String getField() { return field; }
+
+  /** Returns <code>true</code> if the lower endpoint is inclusive */
+  public boolean includesMin() { return minInclusive; }
+  
+  /** Returns <code>true</code> if the upper endpoint is inclusive */
+  public boolean includesMax() { return maxInclusive; }
+
+  //@Override
+  public String toString(final String field) {
+    final StringBuffer sb=new StringBuffer();
+    if (!this.field.equals(field)) sb.append(this.field).append(':');
+    return sb.append(minInclusive ? '[' : '{')
+      .append((min==null) ? "*" : min.toString())
+      .append(" TO ")
+      .append((max==null) ? "*" : max.toString())
+      .append(maxInclusive ? ']' : '}').toString();
+  }
+
+  //@Override
+  public final boolean equals(final Object o) {
+    if (o==this) return true;
+    if (o==null) return false;
+    if (this.getClass().equals(o.getClass())) {
+      AbstractTrieRangeQuery q=(AbstractTrieRangeQuery)o;
+      return (
+        field==q.field &&
+        (q.min == null ? min == null : q.min.equals(min)) &&
+        (q.max == null ? max == null : q.max.equals(max)) &&
+        minInclusive==q.minInclusive &&
+        maxInclusive==q.maxInclusive &&
+        precisionStep==q.precisionStep
+      );
+    }
+    return false;
+  }
+
+  //@Override
+  public final int hashCode() {
+    int hash = field.hashCode() + (precisionStep^0x64365465);
+    if (min!=null) hash += min.hashCode()^0x14fa55fb;
+    if (max!=null) hash += max.hashCode()^0x733fa5fe;
+    return hash+
+      (Boolean.valueOf(minInclusive).hashCode()^0x14fa55fb)+
+      (Boolean.valueOf(maxInclusive).hashCode()^0x733fa5fe);
+  }
+  
+  // TODO: Make this method accessible by *TrieRangeFilter,
+  // can be removed, when moved to core.
+  //@Override
+  protected Filter getFilter() {
+    return super.getFilter();
+  }
+  
+  // members
+  final String field;
+  final int precisionStep;
+  final Number min,max;
+  final boolean minInclusive,maxInclusive;
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeQuery.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java?rev=765618&r1=765617&r2=765618&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java Thu Apr 16 12:58:28 2009
@@ -1,101 +1,61 @@
-package org.apache.lucene.search.trie;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.TermDocs;
-import org.apache.lucene.util.OpenBitSet;
-
-
-/**
- * Implementation of a Lucene {@link Filter} that implements trie-based range filtering for ints/floats.
- * This filter depends on a specific structure of terms in the index that can only be created
- * by indexing via {@link IntTrieTokenStream} methods.
- * For more information, how the algorithm works, see the {@linkplain org.apache.lucene.search.trie package description}.
- */
-public class IntTrieRangeFilter extends AbstractTrieRangeFilter {
-
-  /**
-   * A trie filter for matching trie coded values using the given field name and
-   * the default helper field.
-   * <code>precisionStep</code> must me equal or a multiple of the <code>precisionStep</code>
-   * used for indexing the values.
-   * You can leave the bounds open, by supplying <code>null</code> for <code>min</code> and/or
-   * <code>max</code>. Inclusive/exclusive bounds can also be supplied.
-   * To query float values use the converter {@link TrieUtils#floatToSortableInt}.
-   */
-  public IntTrieRangeFilter(final String field, final int precisionStep,
-    final Integer min, final Integer max, final boolean minInclusive, final boolean maxInclusive
-  ) {
-    super(field,precisionStep,min,max,minInclusive,maxInclusive);
-  }
-
-  /**
-   * Returns a DocIdSet that provides the documents which should be permitted or prohibited in search results.
-   */
-  //@Override
-  public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
-    // calculate the upper and lower bounds respecting the inclusive and null values.
-    int minBound=(this.min==null) ? Integer.MIN_VALUE : (
-      minInclusive ? this.min.intValue() : (this.min.intValue()+1)
-    );
-    int maxBound=(this.max==null) ? Integer.MAX_VALUE : (
-      maxInclusive ? this.max.intValue() : (this.max.intValue()-1)
-    );
-    
-    resetLastNumberOfTerms();
-    if (minBound > maxBound) {
-      // shortcut, no docs will match this
-      return DocIdSet.EMPTY_DOCIDSET;
-    } else {
-      final OpenBitSet bits = new OpenBitSet(reader.maxDoc());
-      final TermDocs termDocs = reader.termDocs();
-      try {
-        TrieUtils.splitIntRange(new TrieUtils.IntRangeBuilder() {
-        
-          //@Override
-          public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
-            try {
-              fillBits(
-                reader, bits, termDocs,
-                minPrefixCoded, maxPrefixCoded
-              );
-            } catch (IOException ioe) {
-              // IntRangeBuilder is not allowed to throw checked exceptions:
-              // wrap as RuntimeException
-              throw new RuntimeException(ioe);
-            }
-          }
-        
-        }, precisionStep, minBound, maxBound);
-      } catch (RuntimeException e) {
-        if (e.getCause() instanceof IOException) throw (IOException)e.getCause();
-        throw e;
-      } finally {
-        termDocs.close();
-      }
-      return bits;
-    }
-  }
-
-}
+package org.apache.lucene.search.trie;
+
+/**
+ * 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.search.Filter; // for javadocs
+import org.apache.lucene.search.MultiTermQueryWrapperFilter;
+
+/**
+ * Implementation of a Lucene {@link Filter} that implements trie-based range filtering for ints/floats.
+ * This filter depends on a specific structure of terms in the index that can only be created
+ * by indexing via {@link IntTrieTokenStream} methods.
+ * For more information, how the algorithm works, see the {@linkplain org.apache.lucene.search.trie package description}.
+ */
+public class IntTrieRangeFilter extends MultiTermQueryWrapperFilter {
+
+  /**
+   * A trie filter for matching trie coded values using the given field name and
+   * the default helper field.
+   * <code>precisionStep</code> must me equal or a multiple of the <code>precisionStep</code>
+   * used for indexing the values.
+   * You can leave the bounds open, by supplying <code>null</code> for <code>min</code> and/or
+   * <code>max</code>. Inclusive/exclusive bounds can also be supplied.
+   * To filter float values use the converter {@link TrieUtils#floatToSortableInt}.
+   */
+  public IntTrieRangeFilter(final String field, final int precisionStep,
+    final Integer min, final Integer max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    super(new IntTrieRangeQuery(field,precisionStep,min,max,minInclusive,maxInclusive));
+  }
+
+  /** Returns the field name for this filter */
+  public String getField() { return ((IntTrieRangeQuery)query).getField(); }
+
+  /** Returns <code>true</code> if the lower endpoint is inclusive */
+  public boolean includesMin() { return ((IntTrieRangeQuery)query).includesMin(); }
+  
+  /** Returns <code>true</code> if the upper endpoint is inclusive */
+  public boolean includesMax() { return ((IntTrieRangeQuery)query).includesMax(); }
+
+  /** Returns the lower value of this range filter */
+  public Integer getMin() { return ((IntTrieRangeQuery)query).getMin(); }
+
+  /** Returns the upper value of this range filter */
+  public Integer getMax() { return ((IntTrieRangeQuery)query).getMax(); }
+
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeQuery.java?rev=765618&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeQuery.java (added)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeQuery.java Thu Apr 16 12:58:28 2009
@@ -0,0 +1,66 @@
+package org.apache.lucene.search.trie;
+
+/**
+ * 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.search.Query;
+
+/**
+ * Implementation of a Lucene {@link Query} that implements trie-based range querying for ints/floats.
+ * This query depends on a specific structure of terms in the index that can only be created
+ * by indexing via {@link IntTrieTokenStream} methods.
+ * <p>The query is in constant score mode per default. With precision steps of &le;4, this
+ * query can be run in conventional boolean rewrite mode without changing the max clause count.
+ * For more information, how the algorithm works, see the {@linkplain org.apache.lucene.search.trie package description}.
+ */
+public class IntTrieRangeQuery extends AbstractTrieRangeQuery {
+
+  /**
+   * A trie query for matching trie coded values using the given field name and
+   * the default helper field.
+   * <code>precisionStep</code> must me equal or a multiple of the <code>precisionStep</code>
+   * used for indexing the values.
+   * You can leave the bounds open, by supplying <code>null</code> for <code>min</code> and/or
+   * <code>max</code>. Inclusive/exclusive bounds can also be supplied.
+   * To query float values use the converter {@link TrieUtils#floatToSortableInt}.
+   */
+  public IntTrieRangeQuery(final String field, final int precisionStep,
+    final Integer min, final Integer max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    super(field,precisionStep,min,max,minInclusive,maxInclusive);
+  }
+
+  //@Override
+  void passRanges(TrieRangeTermEnum enumerator) {
+    // calculate the upper and lower bounds respecting the inclusive and null values.
+    int minBound=(this.min==null) ? Integer.MIN_VALUE : (
+      minInclusive ? this.min.intValue() : (this.min.intValue()+1)
+    );
+    int maxBound=(this.max==null) ? Integer.MAX_VALUE : (
+      maxInclusive ? this.max.intValue() : (this.max.intValue()-1)
+    );
+    
+    TrieUtils.splitIntRange(enumerator.getIntRangeBuilder(), precisionStep, minBound, maxBound);
+  }
+
+  /** Returns the lower value of this range query */
+  public Integer getMin() { return (Integer)min; }
+
+  /** Returns the upper value of this range query */
+  public Integer getMax() { return (Integer)max; }
+  
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeQuery.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieTokenStream.java?rev=765618&r1=765617&r2=765618&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieTokenStream.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieTokenStream.java Thu Apr 16 12:58:28 2009
@@ -1,172 +1,172 @@
-package org.apache.lucene.search.trie;
-
-/**
- * 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.TokenStream;
-import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-
-/**
- * This class provides a {@link TokenStream} for indexing <code>int</code> values
- * that can be queried by {@link IntTrieRangeFilter}. This stream is not intended
- * to be used in analyzers, its more for iterating the different precisions during
- * indexing a specific numeric value.
- * <p>A <code>int</code> value is indexed as multiple string encoded terms, each reduced
- * by zeroing bits from the right. Each value is also prefixed (in the first char) by the
- * <code>shift</code> value (number of bits removed) used during encoding.
- * <p>The number of bits removed from the right for each trie entry is called
- * <code>precisionStep</code> in this API. For comparing the different step values, see the
- * {@linkplain org.apache.lucene.search.trie package description}.
- * <p>The usage pattern is (it is recommened to switch off norms and term frequencies
- * for numeric fields; it does not make sense to have them):
- * <pre>
- *  Field field = new Field(name, new IntTrieTokenStream(value, precisionStep));
- *  field.setOmitNorms(true);
- *  field.setOmitTermFreqAndPositions(true);
- *  document.add(field);
- * </pre>
- * <p>For optimal performance, re-use the TokenStream and Field instance
- * for more than one document:
- * <pre>
- *  <em>// init</em>
- *  TokenStream stream = new IntTrieTokenStream(precisionStep);
- *  Field field = new Field(name, stream);
- *  field.setOmitNorms(true);
- *  field.setOmitTermFreqAndPositions(true);
- *  Document doc = new Document();
- *  document.add(field);
- *  <em>// use this code to index many documents:</em>
- *  stream.setValue(value1)
- *  writer.addDocument(document);
- *  stream.setValue(value2)
- *  writer.addDocument(document);
- *  ...
- * </pre>
- * <p><em>Please note:</em> Token streams are read, when the document is added to index.
- * If you index more than one numeric field, use a separate instance for each.
- * <p>For more information, how trie fields work, see the
- * {@linkplain org.apache.lucene.search.trie package description}.
- */
-public class IntTrieTokenStream extends TokenStream {
-
-  /** The full precision token gets this token type assigned. */
-  public static final String TOKEN_TYPE_FULL_PREC  = "fullPrecTrieInt";
-
-  /** The lower precision tokens gets this token type assigned. */
-  public static final String TOKEN_TYPE_LOWER_PREC = "lowerPrecTrieInt";
-
-  /**
-   * Creates a token stream for indexing <code>value</code> with the given
-   * <code>precisionStep</code>. As instance creating is a major cost,
-   * consider using a {@link #IntTrieTokenStream(int)} instance once for
-   * indexing a large number of documents and assign a value with
-   * {@link #setValue} for each document.
-   * To index float values use the converter {@link TrieUtils#doubleToSortableLong}.
-   */
-  public IntTrieTokenStream(final int value, final int precisionStep) {
-    if (precisionStep<1 || precisionStep>32)
-      throw new IllegalArgumentException("precisionStep may only be 1..32");
-    this.value = value;
-    this.precisionStep = precisionStep;
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-    typeAtt = (TypeAttribute) addAttribute(TypeAttribute.class);
-    posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
-    shiftAtt = (ShiftAttribute) addAttribute(ShiftAttribute.class);
-  }
-  
-  /**
-   * Creates a token stream for indexing values with the given
-   * <code>precisionStep</code>. This stream is initially &quot;empty&quot;
-   * (using a numeric value of 0), assign a value before indexing
-   * each document using {@link #setValue}.
-   */
-  public IntTrieTokenStream(final int precisionStep) {
-    this(0, precisionStep);
-  }
-  
-  /**
-   * Resets the token stream to deliver prefix encoded values
-   * for <code>value</code>. Use this method to index the same
-   * numeric field for a large number of documents and reuse the
-   * current stream instance.
-   * To index float values use the converter {@link TrieUtils#doubleToSortableLong}.
-   */
-  public void setValue(final int value) {
-    this.value = value;
-    reset();
-  }
-  
-  // @Override
-  public void reset() {
-    shift = 0;
-  }
-
-  // @Override
-  public boolean incrementToken() {
-    if (shift>=32) return false;
-    final char[] buffer = termAtt.resizeTermBuffer(TrieUtils.INT_BUF_SIZE);
-    termAtt.setTermLength(TrieUtils.intToPrefixCoded(value, shift, buffer));
-    shiftAtt.setShift(shift);
-    if (shift==0) {
-      typeAtt.setType(TOKEN_TYPE_FULL_PREC);
-      posIncrAtt.setPositionIncrement(1);
-    } else {
-      typeAtt.setType(TOKEN_TYPE_LOWER_PREC);
-      posIncrAtt.setPositionIncrement(0);
-    }
-    shift += precisionStep;
-    return true;
-  }
-
-  // @Override
-  /** @deprecated */
-  public Token next(final Token reusableToken) {
-    if (shift>=32) return null;
-    reusableToken.clear();
-    final char[] buffer = reusableToken.resizeTermBuffer(TrieUtils.INT_BUF_SIZE);
-    reusableToken.setTermLength(TrieUtils.intToPrefixCoded(value, shift, buffer));
-    if (shift==0) {
-      reusableToken.setType(TOKEN_TYPE_FULL_PREC);
-      reusableToken.setPositionIncrement(1);
-    } else {
-      reusableToken.setType(TOKEN_TYPE_LOWER_PREC);
-      reusableToken.setPositionIncrement(0);
-    }
-    shift += precisionStep;
-    return reusableToken;
-  }
-  
-  // @Override
-  public String toString() {
-    final StringBuffer sb = new StringBuffer("(trie-int,value=").append(value);
-    sb.append(",precisionStep=").append(precisionStep).append(')');
-    return sb.toString();
-  }
-
-  // members
-  private final TermAttribute termAtt;
-  private final TypeAttribute typeAtt;
-  private final PositionIncrementAttribute posIncrAtt;
-  private final ShiftAttribute shiftAtt;
-  
-  private int shift = 0;
-  private int value;
-  private final int precisionStep;
-}
+package org.apache.lucene.search.trie;
+
+/**
+ * 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.TokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+
+/**
+ * This class provides a {@link TokenStream} for indexing <code>int</code> values
+ * that can be queried by {@link IntTrieRangeFilter}. This stream is not intended
+ * to be used in analyzers, its more for iterating the different precisions during
+ * indexing a specific numeric value.
+ * <p>A <code>int</code> value is indexed as multiple string encoded terms, each reduced
+ * by zeroing bits from the right. Each value is also prefixed (in the first char) by the
+ * <code>shift</code> value (number of bits removed) used during encoding.
+ * <p>The number of bits removed from the right for each trie entry is called
+ * <code>precisionStep</code> in this API. For comparing the different step values, see the
+ * {@linkplain org.apache.lucene.search.trie package description}.
+ * <p>The usage pattern is (it is recommened to switch off norms and term frequencies
+ * for numeric fields; it does not make sense to have them):
+ * <pre>
+ *  Field field = new Field(name, new IntTrieTokenStream(value, precisionStep));
+ *  field.setOmitNorms(true);
+ *  field.setOmitTermFreqAndPositions(true);
+ *  document.add(field);
+ * </pre>
+ * <p>For optimal performance, re-use the TokenStream and Field instance
+ * for more than one document:
+ * <pre>
+ *  <em>// init</em>
+ *  TokenStream stream = new IntTrieTokenStream(precisionStep);
+ *  Field field = new Field(name, stream);
+ *  field.setOmitNorms(true);
+ *  field.setOmitTermFreqAndPositions(true);
+ *  Document doc = new Document();
+ *  document.add(field);
+ *  <em>// use this code to index many documents:</em>
+ *  stream.setValue(value1)
+ *  writer.addDocument(document);
+ *  stream.setValue(value2)
+ *  writer.addDocument(document);
+ *  ...
+ * </pre>
+ * <p><em>Please note:</em> Token streams are read, when the document is added to index.
+ * If you index more than one numeric field, use a separate instance for each.
+ * <p>For more information, how trie fields work, see the
+ * {@linkplain org.apache.lucene.search.trie package description}.
+ */
+public class IntTrieTokenStream extends TokenStream {
+
+  /** The full precision token gets this token type assigned. */
+  public static final String TOKEN_TYPE_FULL_PREC  = "fullPrecTrieInt";
+
+  /** The lower precision tokens gets this token type assigned. */
+  public static final String TOKEN_TYPE_LOWER_PREC = "lowerPrecTrieInt";
+
+  /**
+   * Creates a token stream for indexing <code>value</code> with the given
+   * <code>precisionStep</code>. As instance creating is a major cost,
+   * consider using a {@link #IntTrieTokenStream(int)} instance once for
+   * indexing a large number of documents and assign a value with
+   * {@link #setValue} for each document.
+   * To index float values use the converter {@link TrieUtils#doubleToSortableLong}.
+   */
+  public IntTrieTokenStream(final int value, final int precisionStep) {
+    if (precisionStep<1 || precisionStep>32)
+      throw new IllegalArgumentException("precisionStep may only be 1..32");
+    this.value = value;
+    this.precisionStep = precisionStep;
+    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
+    typeAtt = (TypeAttribute) addAttribute(TypeAttribute.class);
+    posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
+    shiftAtt = (ShiftAttribute) addAttribute(ShiftAttribute.class);
+  }
+  
+  /**
+   * Creates a token stream for indexing values with the given
+   * <code>precisionStep</code>. This stream is initially &quot;empty&quot;
+   * (using a numeric value of 0), assign a value before indexing
+   * each document using {@link #setValue}.
+   */
+  public IntTrieTokenStream(final int precisionStep) {
+    this(0, precisionStep);
+  }
+  
+  /**
+   * Resets the token stream to deliver prefix encoded values
+   * for <code>value</code>. Use this method to index the same
+   * numeric field for a large number of documents and reuse the
+   * current stream instance.
+   * To index float values use the converter {@link TrieUtils#doubleToSortableLong}.
+   */
+  public void setValue(final int value) {
+    this.value = value;
+    reset();
+  }
+  
+  // @Override
+  public void reset() {
+    shift = 0;
+  }
+
+  // @Override
+  public boolean incrementToken() {
+    if (shift>=32) return false;
+    final char[] buffer = termAtt.resizeTermBuffer(TrieUtils.INT_BUF_SIZE);
+    termAtt.setTermLength(TrieUtils.intToPrefixCoded(value, shift, buffer));
+    shiftAtt.setShift(shift);
+    if (shift==0) {
+      typeAtt.setType(TOKEN_TYPE_FULL_PREC);
+      posIncrAtt.setPositionIncrement(1);
+    } else {
+      typeAtt.setType(TOKEN_TYPE_LOWER_PREC);
+      posIncrAtt.setPositionIncrement(0);
+    }
+    shift += precisionStep;
+    return true;
+  }
+
+  // @Override
+  /** @deprecated */
+  public Token next(final Token reusableToken) {
+    if (shift>=32) return null;
+    reusableToken.clear();
+    final char[] buffer = reusableToken.resizeTermBuffer(TrieUtils.INT_BUF_SIZE);
+    reusableToken.setTermLength(TrieUtils.intToPrefixCoded(value, shift, buffer));
+    if (shift==0) {
+      reusableToken.setType(TOKEN_TYPE_FULL_PREC);
+      reusableToken.setPositionIncrement(1);
+    } else {
+      reusableToken.setType(TOKEN_TYPE_LOWER_PREC);
+      reusableToken.setPositionIncrement(0);
+    }
+    shift += precisionStep;
+    return reusableToken;
+  }
+  
+  // @Override
+  public String toString() {
+    final StringBuffer sb = new StringBuffer("(trie-int,value=").append(value);
+    sb.append(",precisionStep=").append(precisionStep).append(')');
+    return sb.toString();
+  }
+
+  // members
+  private final TermAttribute termAtt;
+  private final TypeAttribute typeAtt;
+  private final PositionIncrementAttribute posIncrAtt;
+  private final ShiftAttribute shiftAtt;
+  
+  private int shift = 0;
+  private int value;
+  private final int precisionStep;
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieTokenStream.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java?rev=765618&r1=765617&r2=765618&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java Thu Apr 16 12:58:28 2009
@@ -1,101 +1,61 @@
-package org.apache.lucene.search.trie;
-
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.TermDocs;
-import org.apache.lucene.util.OpenBitSet;
-
-
-/**
- * Implementation of a Lucene {@link Filter} that implements trie-based range filtering for longs/doubles.
- * This filter depends on a specific structure of terms in the index that can only be created
- * by indexing via {@link LongTrieTokenStream} methods.
- * For more information, how the algorithm works, see the {@linkplain org.apache.lucene.search.trie package description}.
- */
-public class LongTrieRangeFilter extends AbstractTrieRangeFilter {
-
-  /**
-   * A trie filter for matching trie coded values using the given field name and
-   * the default helper field.
-   * <code>precisionStep</code> must me equal or a multiple of the <code>precisionStep</code>
-   * used for indexing the values.
-   * You can leave the bounds open, by supplying <code>null</code> for <code>min</code> and/or
-   * <code>max</code>. Inclusive/exclusive bounds can also be supplied.
-   * To query double values use the converter {@link TrieUtils#doubleToSortableLong}.
-   */
-  public LongTrieRangeFilter(final String field, final int precisionStep,
-    final Long min, final Long max, final boolean minInclusive, final boolean maxInclusive
-  ) {
-    super(field,precisionStep,min,max,minInclusive,maxInclusive);
-  }
-
-  /**
-   * Returns a DocIdSet that provides the documents which should be permitted or prohibited in search results.
-   */
-  //@Override
-  public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
-    // calculate the upper and lower bounds respecting the inclusive and null values.
-    long minBound=(this.min==null) ? Long.MIN_VALUE : (
-      minInclusive ? this.min.longValue() : (this.min.longValue()+1L)
-    );
-    long maxBound=(this.max==null) ? Long.MAX_VALUE : (
-      maxInclusive ? this.max.longValue() : (this.max.longValue()-1L)
-    );
-    
-    resetLastNumberOfTerms();
-    if (minBound > maxBound) {
-      // shortcut, no docs will match this
-      return DocIdSet.EMPTY_DOCIDSET;
-    } else {
-      final OpenBitSet bits = new OpenBitSet(reader.maxDoc());
-      final TermDocs termDocs = reader.termDocs();
-      try {
-        TrieUtils.splitLongRange(new TrieUtils.LongRangeBuilder() {
-        
-          //@Override
-          public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
-            try {
-              fillBits(
-                reader, bits, termDocs,
-                minPrefixCoded, maxPrefixCoded
-              );
-            } catch (IOException ioe) {
-              // LongRangeBuilder is not allowed to throw checked exceptions:
-              // wrap as RuntimeException
-              throw new RuntimeException(ioe);
-            }
-          }
-        
-        }, precisionStep, minBound, maxBound);
-      } catch (RuntimeException e) {
-        if (e.getCause() instanceof IOException) throw (IOException)e.getCause();
-        throw e;
-      } finally {
-        termDocs.close();
-      }
-      return bits;
-    }
-  }
-
-}
+package org.apache.lucene.search.trie;
+
+/**
+ * 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.search.Filter; // for javadocs
+import org.apache.lucene.search.MultiTermQueryWrapperFilter;
+
+/**
+ * Implementation of a Lucene {@link Filter} that implements trie-based range filtering for longs/doubles.
+ * This filter depends on a specific structure of terms in the index that can only be created
+ * by indexing via {@link LongTrieTokenStream} methods.
+ * For more information, how the algorithm works, see the {@linkplain org.apache.lucene.search.trie package description}.
+ */
+public class LongTrieRangeFilter extends MultiTermQueryWrapperFilter {
+
+  /**
+   * A trie filter for matching trie coded values using the given field name and
+   * the default helper field.
+   * <code>precisionStep</code> must me equal or a multiple of the <code>precisionStep</code>
+   * used for indexing the values.
+   * You can leave the bounds open, by supplying <code>null</code> for <code>min</code> and/or
+   * <code>max</code>. Inclusive/exclusive bounds can also be supplied.
+   * To filter double values use the converter {@link TrieUtils#doubleToSortableLong}.
+   */
+  public LongTrieRangeFilter(final String field, final int precisionStep,
+    final Long min, final Long max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    super(new LongTrieRangeQuery(field,precisionStep,min,max,minInclusive,maxInclusive));
+  }
+
+  /** Returns the field name for this filter */
+  public String getField() { return ((LongTrieRangeQuery)query).getField(); }
+
+  /** Returns <code>true</code> if the lower endpoint is inclusive */
+  public boolean includesMin() { return ((LongTrieRangeQuery)query).includesMin(); }
+  
+  /** Returns <code>true</code> if the upper endpoint is inclusive */
+  public boolean includesMax() { return ((LongTrieRangeQuery)query).includesMax(); }
+
+  /** Returns the lower value of this range filter */
+  public Long getMin() { return ((LongTrieRangeQuery)query).getMin(); }
+
+  /** Returns the upper value of this range filter */
+  public Long getMax() { return ((LongTrieRangeQuery)query).getMax(); }
+  
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeQuery.java?rev=765618&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeQuery.java (added)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeQuery.java Thu Apr 16 12:58:28 2009
@@ -0,0 +1,66 @@
+package org.apache.lucene.search.trie;
+
+/**
+ * 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.search.Query;
+
+/**
+ * Implementation of a Lucene {@link Query} that implements trie-based range querying for longs/doubles.
+ * This query depends on a specific structure of terms in the index that can only be created
+ * by indexing via {@link LongTrieTokenStream} methods.
+ * <p>The query is in constant score mode per default. With precision steps of &le;4, this
+ * query can be run in conventional boolean rewrite mode without changing the max clause count.
+ * For more information, how the algorithm works, see the {@linkplain org.apache.lucene.search.trie package description}.
+ */
+public class LongTrieRangeQuery extends AbstractTrieRangeQuery {
+
+  /**
+   * A trie query for matching trie coded values using the given field name and
+   * the default helper field.
+   * <code>precisionStep</code> must me equal or a multiple of the <code>precisionStep</code>
+   * used for indexing the values.
+   * You can leave the bounds open, by supplying <code>null</code> for <code>min</code> and/or
+   * <code>max</code>. Inclusive/exclusive bounds can also be supplied.
+   * To query double values use the converter {@link TrieUtils#doubleToSortableLong}.
+   */
+  public LongTrieRangeQuery(final String field, final int precisionStep,
+    final Long min, final Long max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    super(field,precisionStep,min,max,minInclusive,maxInclusive);
+  }
+
+  //@Override
+  void passRanges(TrieRangeTermEnum enumerator) {
+    // calculate the upper and lower bounds respecting the inclusive and null values.
+    long minBound=(this.min==null) ? Long.MIN_VALUE : (
+      minInclusive ? this.min.longValue() : (this.min.longValue()+1L)
+    );
+    long maxBound=(this.max==null) ? Long.MAX_VALUE : (
+      maxInclusive ? this.max.longValue() : (this.max.longValue()-1L)
+    );
+    
+    TrieUtils.splitLongRange(enumerator.getLongRangeBuilder(), precisionStep, minBound, maxBound);
+  }
+
+  /** Returns the lower value of this range query */
+  public Long getMin() { return (Long)min; }
+
+  /** Returns the upper value of this range query */
+  public Long getMax() { return (Long)max; }
+  
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeQuery.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieTokenStream.java?rev=765618&r1=765617&r2=765618&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieTokenStream.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieTokenStream.java Thu Apr 16 12:58:28 2009
@@ -1,172 +1,172 @@
-package org.apache.lucene.search.trie;
-
-/**
- * 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.TokenStream;
-import org.apache.lucene.analysis.Token;
-import org.apache.lucene.analysis.tokenattributes.TermAttribute;
-import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-
-/**
- * This class provides a {@link TokenStream} for indexing <code>long</code> values
- * that can be queried by {@link LongTrieRangeFilter}. This stream is not intended
- * to be used in analyzers, its more for iterating the different precisions during
- * indexing a specific numeric value.
- * <p>A <code>long</code> value is indexed as multiple string encoded terms, each reduced
- * by zeroing bits from the right. Each value is also prefixed (in the first char) by the
- * <code>shift</code> value (number of bits removed) used during encoding.
- * <p>The number of bits removed from the right for each trie entry is called
- * <code>precisionStep</code> in this API. For comparing the different step values, see the
- * {@linkplain org.apache.lucene.search.trie package description}.
- * <p>The usage pattern is (it is recommened to switch off norms and term frequencies
- * for numeric fields; it does not make sense to have them):
- * <pre>
- *  Field field = new Field(name, new LongTrieTokenStream(value, precisionStep));
- *  field.setOmitNorms(true);
- *  field.setOmitTermFreqAndPositions(true);
- *  document.add(field);
- * </pre>
- * <p>For optimal performance, re-use the TokenStream and Field instance
- * for more than one document:
- * <pre>
- *  <em>// init</em>
- *  TokenStream stream = new LongTrieTokenStream(precisionStep);
- *  Field field = new Field(name, stream);
- *  field.setOmitNorms(true);
- *  field.setOmitTermFreqAndPositions(true);
- *  Document doc = new Document();
- *  document.add(field);
- *  <em>// use this code to index many documents:</em>
- *  stream.setValue(value1)
- *  writer.addDocument(document);
- *  stream.setValue(value2)
- *  writer.addDocument(document);
- *  ...
- * </pre>
- * <p><em>Please note:</em> Token streams are read, when the document is added to index.
- * If you index more than one numeric field, use a separate instance for each.
- * <p>For more information, how trie fields work, see the
- * {@linkplain org.apache.lucene.search.trie package description}.
- */
-public class LongTrieTokenStream extends TokenStream {
-
-  /** The full precision token gets this token type assigned. */
-  public static final String TOKEN_TYPE_FULL_PREC  = "fullPrecTrieLong";
-
-  /** The lower precision tokens gets this token type assigned. */
-  public static final String TOKEN_TYPE_LOWER_PREC = "lowerPrecTrieLong";
-
-  /**
-   * Creates a token stream for indexing <code>value</code> with the given
-   * <code>precisionStep</code>. As instance creating is a major cost,
-   * consider using a {@link #LongTrieTokenStream(int)} instance once for
-   * indexing a large number of documents and assign a value with
-   * {@link #setValue} for each document.
-   * To index double values use the converter {@link TrieUtils#doubleToSortableLong}.
-   */
-  public LongTrieTokenStream(final long value, final int precisionStep) {
-    if (precisionStep<1 || precisionStep>64)
-      throw new IllegalArgumentException("precisionStep may only be 1..64");
-    this.value = value;
-    this.precisionStep = precisionStep;
-    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
-    typeAtt = (TypeAttribute) addAttribute(TypeAttribute.class);
-    posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
-    shiftAtt = (ShiftAttribute) addAttribute(ShiftAttribute.class);
-  }
-  
-  /**
-   * Creates a token stream for indexing values with the given
-   * <code>precisionStep</code>. This stream is initially &quot;empty&quot;
-   * (using a numeric value of 0), assign a value before indexing
-   * each document using {@link #setValue}.
-   */
-  public LongTrieTokenStream(final int precisionStep) {
-    this(0L, precisionStep);
-  }
-
-  /**
-   * Resets the token stream to deliver prefix encoded values
-   * for <code>value</code>. Use this method to index the same
-   * numeric field for a large number of documents and reuse the
-   * current stream instance.
-   * To index double values use the converter {@link TrieUtils#doubleToSortableLong}.
-   */
-  public void setValue(final long value) {
-    this.value = value;
-    reset();
-  }
-  
-  // @Override
-  public void reset() {
-    shift = 0;
-  }
-
-  // @Override
-  public boolean incrementToken() {
-    if (shift>=64) return false;
-    final char[] buffer = termAtt.resizeTermBuffer(TrieUtils.LONG_BUF_SIZE);
-    termAtt.setTermLength(TrieUtils.longToPrefixCoded(value, shift, buffer));
-    shiftAtt.setShift(shift);
-    if (shift==0) {
-      typeAtt.setType(TOKEN_TYPE_FULL_PREC);
-      posIncrAtt.setPositionIncrement(1);
-    } else {
-      typeAtt.setType(TOKEN_TYPE_LOWER_PREC);
-      posIncrAtt.setPositionIncrement(0);
-    }
-    shift += precisionStep;
-    return true;
-  }
-
-  // @Override
-  /** @deprecated */
-  public Token next(final Token reusableToken) {
-    if (shift>=64) return null;
-    reusableToken.clear();
-    final char[] buffer = reusableToken.resizeTermBuffer(TrieUtils.LONG_BUF_SIZE);
-    reusableToken.setTermLength(TrieUtils.longToPrefixCoded(value, shift, buffer));
-    if (shift==0) {
-      reusableToken.setType(TOKEN_TYPE_FULL_PREC);
-      reusableToken.setPositionIncrement(1);
-    } else {
-      reusableToken.setType(TOKEN_TYPE_LOWER_PREC);
-      reusableToken.setPositionIncrement(0);
-    }
-    shift += precisionStep;
-    return reusableToken;
-  }
-  
-  // @Override
-  public String toString() {
-    final StringBuffer sb = new StringBuffer("(trie-long,value=").append(value);
-    sb.append(",precisionStep=").append(precisionStep).append(')');
-    return sb.toString();
-  }
-
-  // members
-  private final TermAttribute termAtt;
-  private final TypeAttribute typeAtt;
-  private final PositionIncrementAttribute posIncrAtt;
-  private final ShiftAttribute shiftAtt;
-  
-  private int shift = 0;
-  private long value;
-  private final int precisionStep;
-}
+package org.apache.lucene.search.trie;
+
+/**
+ * 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.TokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.tokenattributes.TermAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+
+/**
+ * This class provides a {@link TokenStream} for indexing <code>long</code> values
+ * that can be queried by {@link LongTrieRangeFilter}. This stream is not intended
+ * to be used in analyzers, its more for iterating the different precisions during
+ * indexing a specific numeric value.
+ * <p>A <code>long</code> value is indexed as multiple string encoded terms, each reduced
+ * by zeroing bits from the right. Each value is also prefixed (in the first char) by the
+ * <code>shift</code> value (number of bits removed) used during encoding.
+ * <p>The number of bits removed from the right for each trie entry is called
+ * <code>precisionStep</code> in this API. For comparing the different step values, see the
+ * {@linkplain org.apache.lucene.search.trie package description}.
+ * <p>The usage pattern is (it is recommened to switch off norms and term frequencies
+ * for numeric fields; it does not make sense to have them):
+ * <pre>
+ *  Field field = new Field(name, new LongTrieTokenStream(value, precisionStep));
+ *  field.setOmitNorms(true);
+ *  field.setOmitTermFreqAndPositions(true);
+ *  document.add(field);
+ * </pre>
+ * <p>For optimal performance, re-use the TokenStream and Field instance
+ * for more than one document:
+ * <pre>
+ *  <em>// init</em>
+ *  TokenStream stream = new LongTrieTokenStream(precisionStep);
+ *  Field field = new Field(name, stream);
+ *  field.setOmitNorms(true);
+ *  field.setOmitTermFreqAndPositions(true);
+ *  Document doc = new Document();
+ *  document.add(field);
+ *  <em>// use this code to index many documents:</em>
+ *  stream.setValue(value1)
+ *  writer.addDocument(document);
+ *  stream.setValue(value2)
+ *  writer.addDocument(document);
+ *  ...
+ * </pre>
+ * <p><em>Please note:</em> Token streams are read, when the document is added to index.
+ * If you index more than one numeric field, use a separate instance for each.
+ * <p>For more information, how trie fields work, see the
+ * {@linkplain org.apache.lucene.search.trie package description}.
+ */
+public class LongTrieTokenStream extends TokenStream {
+
+  /** The full precision token gets this token type assigned. */
+  public static final String TOKEN_TYPE_FULL_PREC  = "fullPrecTrieLong";
+
+  /** The lower precision tokens gets this token type assigned. */
+  public static final String TOKEN_TYPE_LOWER_PREC = "lowerPrecTrieLong";
+
+  /**
+   * Creates a token stream for indexing <code>value</code> with the given
+   * <code>precisionStep</code>. As instance creating is a major cost,
+   * consider using a {@link #LongTrieTokenStream(int)} instance once for
+   * indexing a large number of documents and assign a value with
+   * {@link #setValue} for each document.
+   * To index double values use the converter {@link TrieUtils#doubleToSortableLong}.
+   */
+  public LongTrieTokenStream(final long value, final int precisionStep) {
+    if (precisionStep<1 || precisionStep>64)
+      throw new IllegalArgumentException("precisionStep may only be 1..64");
+    this.value = value;
+    this.precisionStep = precisionStep;
+    termAtt = (TermAttribute) addAttribute(TermAttribute.class);
+    typeAtt = (TypeAttribute) addAttribute(TypeAttribute.class);
+    posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
+    shiftAtt = (ShiftAttribute) addAttribute(ShiftAttribute.class);
+  }
+  
+  /**
+   * Creates a token stream for indexing values with the given
+   * <code>precisionStep</code>. This stream is initially &quot;empty&quot;
+   * (using a numeric value of 0), assign a value before indexing
+   * each document using {@link #setValue}.
+   */
+  public LongTrieTokenStream(final int precisionStep) {
+    this(0L, precisionStep);
+  }
+
+  /**
+   * Resets the token stream to deliver prefix encoded values
+   * for <code>value</code>. Use this method to index the same
+   * numeric field for a large number of documents and reuse the
+   * current stream instance.
+   * To index double values use the converter {@link TrieUtils#doubleToSortableLong}.
+   */
+  public void setValue(final long value) {
+    this.value = value;
+    reset();
+  }
+  
+  // @Override
+  public void reset() {
+    shift = 0;
+  }
+
+  // @Override
+  public boolean incrementToken() {
+    if (shift>=64) return false;
+    final char[] buffer = termAtt.resizeTermBuffer(TrieUtils.LONG_BUF_SIZE);
+    termAtt.setTermLength(TrieUtils.longToPrefixCoded(value, shift, buffer));
+    shiftAtt.setShift(shift);
+    if (shift==0) {
+      typeAtt.setType(TOKEN_TYPE_FULL_PREC);
+      posIncrAtt.setPositionIncrement(1);
+    } else {
+      typeAtt.setType(TOKEN_TYPE_LOWER_PREC);
+      posIncrAtt.setPositionIncrement(0);
+    }
+    shift += precisionStep;
+    return true;
+  }
+
+  // @Override
+  /** @deprecated */
+  public Token next(final Token reusableToken) {
+    if (shift>=64) return null;
+    reusableToken.clear();
+    final char[] buffer = reusableToken.resizeTermBuffer(TrieUtils.LONG_BUF_SIZE);
+    reusableToken.setTermLength(TrieUtils.longToPrefixCoded(value, shift, buffer));
+    if (shift==0) {
+      reusableToken.setType(TOKEN_TYPE_FULL_PREC);
+      reusableToken.setPositionIncrement(1);
+    } else {
+      reusableToken.setType(TOKEN_TYPE_LOWER_PREC);
+      reusableToken.setPositionIncrement(0);
+    }
+    shift += precisionStep;
+    return reusableToken;
+  }
+  
+  // @Override
+  public String toString() {
+    final StringBuffer sb = new StringBuffer("(trie-long,value=").append(value);
+    sb.append(",precisionStep=").append(precisionStep).append(')');
+    return sb.toString();
+  }
+
+  // members
+  private final TermAttribute termAtt;
+  private final TypeAttribute typeAtt;
+  private final PositionIncrementAttribute posIncrAtt;
+  private final ShiftAttribute shiftAtt;
+  
+  private int shift = 0;
+  private long value;
+  private final int precisionStep;
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieTokenStream.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/ShiftAttribute.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/ShiftAttribute.java?rev=765618&r1=765617&r2=765618&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/ShiftAttribute.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/ShiftAttribute.java Thu Apr 16 12:58:28 2009
@@ -1,70 +1,70 @@
-package org.apache.lucene.search.trie;
-
-/**
- * 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.util.Attribute;
-
-import java.io.Serializable;
-
-/**
- * This attribute is updated by {@link IntTrieTokenStream} and {@link LongTrieTokenStream}
- * to the shift value of the current prefix-encoded token.
- * It may be used by filters or consumers to e.g. distribute the values to various fields.
- */
-public final class ShiftAttribute extends Attribute implements Cloneable, Serializable {
-  private int shift = 0;
-  
-  /**
-   * Returns the shift value of the current prefix encoded token.
-   */
-  public int getShift() {
-    return shift;
-  }
-
-  /**
-   * Sets the shift value.
-   */
-  public void setShift(final int shift) {
-    this.shift = shift;
-  }
-  
-  public void clear() {
-    shift = 0;
-  }
-
-  public String toString() {
-    return "shift=" + shift;
-  }
-
-  public boolean equals(Object other) {
-    if (this == other) return true;
-    if (other instanceof ShiftAttribute) {
-      return ((ShiftAttribute) other).shift == shift;
-    }
-    return false;
-  }
-
-  public int hashCode() {
-    return shift;
-  }
-  
-  public void copyTo(Attribute target) {
-    final ShiftAttribute t = (ShiftAttribute) target;
-    t.setShift(shift);
-  }
-}
+package org.apache.lucene.search.trie;
+
+/**
+ * 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.util.Attribute;
+
+import java.io.Serializable;
+
+/**
+ * This attribute is updated by {@link IntTrieTokenStream} and {@link LongTrieTokenStream}
+ * to the shift value of the current prefix-encoded token.
+ * It may be used by filters or consumers to e.g. distribute the values to various fields.
+ */
+public final class ShiftAttribute extends Attribute implements Cloneable, Serializable {
+  private int shift = 0;
+  
+  /**
+   * Returns the shift value of the current prefix encoded token.
+   */
+  public int getShift() {
+    return shift;
+  }
+
+  /**
+   * Sets the shift value.
+   */
+  public void setShift(final int shift) {
+    this.shift = shift;
+  }
+  
+  public void clear() {
+    shift = 0;
+  }
+
+  public String toString() {
+    return "shift=" + shift;
+  }
+
+  public boolean equals(Object other) {
+    if (this == other) return true;
+    if (other instanceof ShiftAttribute) {
+      return ((ShiftAttribute) other).shift == shift;
+    }
+    return false;
+  }
+
+  public int hashCode() {
+    return shift;
+  }
+  
+  public void copyTo(Attribute target) {
+    final ShiftAttribute t = (ShiftAttribute) target;
+    t.setShift(shift);
+  }
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/ShiftAttribute.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeTermEnum.java?rev=765618&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeTermEnum.java (added)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeTermEnum.java Thu Apr 16 12:58:28 2009
@@ -0,0 +1,140 @@
+package org.apache.lucene.search.trie;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+import org.apache.lucene.search.FilteredTermEnum;
+import org.apache.lucene.search.MultiTermQuery; // for javadocs
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+
+/**
+ * Subclass of FilteredTermEnum for enumerating all terms that match the
+ * sub-ranges for trie range queries.
+ * <p>
+ * WARNING: Term enumerations is not guaranteed to be always ordered by
+ * {@link Term#compareTo}.
+ * The ordering depends on how {@link TrieUtils#splitLongRange} and
+ * {@link TrieUtils#splitIntRange} generates the sub-ranges. For
+ * the {@link MultiTermQuery} ordering is not relevant.
+ */
+final class TrieRangeTermEnum extends FilteredTermEnum {
+
+  private final AbstractTrieRangeQuery query;
+  private final IndexReader reader;
+  private final LinkedList/*<String>*/ rangeBounds = new LinkedList/*<String>*/();
+  private String currentUpperBound = null;
+
+  TrieRangeTermEnum(AbstractTrieRangeQuery query, IndexReader reader) {
+    this.query = query;
+    this.reader = reader;
+  }
+
+  /** Returns a range builder that must be used to feed in the sub-ranges. */
+  TrieUtils.IntRangeBuilder getIntRangeBuilder() {
+    return new TrieUtils.IntRangeBuilder() {
+      //@Override
+      public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
+        rangeBounds.add(minPrefixCoded);
+        rangeBounds.add(maxPrefixCoded);
+      }
+    };
+  }
+
+  /** Returns a range builder that must be used to feed in the sub-ranges. */
+  TrieUtils.LongRangeBuilder getLongRangeBuilder() {
+    return new TrieUtils.LongRangeBuilder() {
+      //@Override
+      public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
+        rangeBounds.add(minPrefixCoded);
+        rangeBounds.add(maxPrefixCoded);
+      }
+    };
+  }
+  
+  /** After feeding the range builder call this method to initialize the enum. */
+  void init() throws IOException {
+    next();
+  }
+
+  //@Override
+  public float difference() {
+    return 1.0f;
+  }
+  
+  /** this is a dummy, it is not used by this class. */
+  //@Override
+  protected boolean endEnum() {
+    assert false; // should never be called
+    return (currentTerm != null);
+  }
+
+  /**
+   * Compares if current upper bound is reached,
+   * this also updates the term count for statistics.
+   * In contrast to {@link FilteredTermEnum}, a return value
+   * of <code>false</code> ends iterating the current enum
+   * and forwards to the next sub-range.
+   */
+  //@Override
+  protected boolean termCompare(Term term) {
+    return (term.field() == query.field && term.text().compareTo(currentUpperBound) <= 0);
+  }
+  
+  /** Increments the enumeration to the next element.  True if one exists. */
+  //@Override
+  public boolean next() throws IOException {
+    // if a current term exists, the actual enum is initialized:
+    // try change to next term, if no such term exists, fall-through
+    if (currentTerm != null) {
+      assert actualEnum!=null;
+      if (actualEnum.next()) {
+        currentTerm = actualEnum.term();
+        if (termCompare(currentTerm)) return true;
+      }
+    }
+    // if all above fails, we go forward to the next enum,
+    // if one is available
+    currentTerm = null;
+    if (rangeBounds.size() < 2) return false;
+    // close the current enum and read next bounds
+    if (actualEnum != null) {
+      actualEnum.close();
+      actualEnum = null;
+    }
+    final String lowerBound = (String)rangeBounds.removeFirst();
+    this.currentUpperBound = (String)rangeBounds.removeFirst();
+    // this call recursively uses next(), if no valid term in
+    // next enum found.
+    // if this behavior is changed/modified in the superclass,
+    // this enum will not work anymore!
+    setEnum(reader.terms(new Term(query.field, lowerBound)));
+    return (currentTerm != null);
+  }
+
+  /** Closes the enumeration to further activity, freeing resources.  */
+  //@Override
+  public void close() throws IOException {
+    rangeBounds.clear();
+    currentUpperBound = null;
+    super.close();
+  }
+
+}

Propchange: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeTermEnum.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/package.html
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/package.html?rev=765618&r1=765617&r2=765618&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/package.html (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/package.html Thu Apr 16 12:58:28 2009
@@ -106,14 +106,14 @@
 <h3>Searching</h3>
 
 <p>The numeric index fields you prepared in this way can be searched by
-{@link org.apache.lucene.search.trie.LongTrieRangeFilter} or {@link org.apache.lucene.search.trie.IntTrieRangeFilter}:</p>
+{@link org.apache.lucene.search.trie.LongTrieRangeQuery} or {@link org.apache.lucene.search.trie.IntTrieRangeQuery}:</p>
 
 <pre>
   <em>// Java 1.4, because Long.valueOf(long) is not available:</em>
-  Query q = new LongTrieRangeFilter("exampleLong", precisionStep, new Long(123L), new Long(999999L), true, true).asQuery();
+  Query q = new LongTrieRangeQuery("exampleLong", precisionStep, new Long(123L), new Long(999999L), true, true);
   
   <em>// OR, Java 1.5, using autoboxing:</em>
-  Query q = new LongTrieRangeFilter("exampleLong", precisionStep, 123L, 999999L, true, true).asQuery();
+  Query q = new LongTrieRangeQuery("exampleLong", precisionStep, 123L, 999999L, true, true);
   
   <em>// execute the search, as usual:</em>
   TopDocs docs = searcher.search(q, 10);
@@ -132,7 +132,7 @@
 that the old {@link org.apache.lucene.search.RangeQuery} (with raised 
 {@link org.apache.lucene.search.BooleanQuery} clause count) took about 30-40 secs to complete,
 {@link org.apache.lucene.search.ConstantScoreRangeQuery} took 5 secs and executing
-{@link org.apache.lucene.search.trie.LongTrieRangeFilter}<code>.asQuery()</code> took &lt;100ms to
+{@link org.apache.lucene.search.trie.LongTrieRangeQuery} took &lt;100ms to
 complete (on an Opteron64 machine, Java 1.5, 8 bit precision step).
 This query type was developed for a geographic portal, where the performance for
 e.g. bounding boxes or exact date/time stamps is important.</p>