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/02/13 19:27:02 UTC

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

Author: uschindler
Date: Fri Feb 13 18:27:01 2009
New Revision: 744207

URL: http://svn.apache.org/viewvc?rev=744207&view=rev
Log:
LUCENE-1470: New implementation using encoding of TrieUtils and TrieRangeFilter that now also supports 32bit and 64bit fields

Added:
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeFilter.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.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
      - copied, changed from r742792, lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestTrieRangeQuery.java
Removed:
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeFilter.java
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieRangeQuery.java
    lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestTrieRangeQuery.java
Modified:
    lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieUtils.java
    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/TestTrieUtils.java

Added: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeFilter.java?rev=744207&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeFilter.java (added)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/AbstractTrieRangeFilter.java Fri Feb 13 18:27:01 2009
@@ -0,0 +1,160 @@
+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.Arrays;
+
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.TermDocs;
+import org.apache.lucene.index.TermEnum;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.ToStringUtils;
+
+
+abstract class AbstractTrieRangeFilter extends Filter {
+
+  AbstractTrieRangeFilter(final String[] fields, final int precisionStep,
+    Number min, Number max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    this.fields=(String[])fields.clone();
+    this.precisionStep=precisionStep;
+    this.min=min;
+    this.max=max;
+    this.minInclusive=minInclusive;
+    this.maxInclusive=maxInclusive;
+  }
+
+  //@Override
+  public String toString() {
+    return toString(null);
+  }
+
+  public String toString(final String field) {
+    final StringBuffer sb=new StringBuffer();
+    if (!this.fields[0].equals(field)) sb.append(this.fields[0]).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())) {
+      AbstractTrieRangeFilter q=(AbstractTrieRangeFilter)o;
+      return (
+        Arrays.equals(fields,q.fields) &&
+        (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=Arrays.asList(fields).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);
+  }
+  
+  /**
+   * Expert: Return the number of terms visited during the last execution of {@link #getDocIdSet}.
+   * This may be used for performance comparisons of different trie variants and their effectiveness.
+   * This method is not thread safe, be sure to only call it when no query is running!
+   * @throws IllegalStateException if {@link #getDocIdSet} was not yet executed.
+   */
+  public int getLastNumberOfTerms() {
+    if (lastNumberOfTerms < 0) throw new IllegalStateException();
+    return lastNumberOfTerms;
+  }
+  
+  void resetLastNumberOfTerms() {
+    lastNumberOfTerms=0;
+  }
+
+  /** Returns this range filter as a query.
+   * Using this method, it is possible to create a Query using <code>new {Long|Int}TrieRangeFilter(....).asQuery()</code>.
+   * This is a synonym for wrapping with a {@link ConstantScoreQuery},
+   * but this query returns a better <code>toString()</code> variant.
+   */
+  public Query asQuery() {
+    return new ConstantScoreQuery(this) {
+    
+      /** this instance return a nicer String variant than the original {@link ConstantScoreQuery} */
+      //@Override
+      public String toString(final String field) {
+        // return a more convenient representation of this query than ConstantScoreQuery does:
+        return ((AbstractTrieRangeFilter) filter).toString(field)+ToStringUtils.boost(getBoost());
+      }
+
+    };
+  }
+  
+  void fillBits(
+    final IndexReader reader,
+    final OpenBitSet bits, final TermDocs termDocs,
+    String field,
+    final String lowerTerm, final String upperTerm
+  ) throws IOException {
+    final int len=lowerTerm.length();
+    assert upperTerm.length()==len;
+    field=field.intern();
+    
+    // find the docs
+    final TermEnum enumerator = reader.terms(new Term(field, lowerTerm));
+    try {
+      do {
+        final Term term = enumerator.term();
+        if (term!=null && term.field()==field) {
+          // break out when upperTerm reached or length of term is different
+          final String t=term.text();
+          if (len!=t.length() || t.compareTo(upperTerm)>0) break;
+          // we have a good term, find the docs
+          lastNumberOfTerms++;
+          termDocs.seek(enumerator);
+          while (termDocs.next()) bits.set(termDocs.doc());
+        } else break;
+      } while (enumerator.next());
+    } finally {
+      enumerator.close();
+    }
+  }
+  
+  // members
+  final String[] fields;
+  final int precisionStep;
+  final Number min,max;
+  final boolean minInclusive,maxInclusive;
+  
+  private int lastNumberOfTerms=-1;
+}

Added: 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=744207&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java (added)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/IntTrieRangeFilter.java Fri Feb 13 18:27:01 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 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 {@link TrieUtils} 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}.
+   * <p>This is the counterpart to {@link TrieUtils#addIndexedFields(Document,String,String[])}.
+   * <p><b>This is the recommended usage of TrieUtils/IntTrieRangeFilter.</b>
+   */
+  public IntTrieRangeFilter(final String field, final int precisionStep,
+    final Integer min, final Integer max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    this(
+      new String[]{field, field+TrieUtils.LOWER_PRECISION_FIELD_NAME_SUFFIX},
+      precisionStep,min,max,minInclusive,maxInclusive
+    );
+  }
+  
+  /**
+   * Expert: A trie filter for matching trie coded values using the given field names.
+   * You can specify the main and helper field name, that was used to idex the values.
+   * <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}.
+   * <p>This is the counterpart to {@link TrieUtils#addIndexedFields(Document,String,String,String[])}.
+   */
+  public IntTrieRangeFilter(final String field, final String lowerPrecisionField, final int precisionStep,
+    final Integer min, final Integer max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    this(new String[]{field, lowerPrecisionField},precisionStep,min,max,minInclusive,maxInclusive);
+  }
+
+  /**
+   * Expert: A trie filter for matching trie coded values
+   * using the given field names. If the array of field names is shorter than the
+   * trieCoded one, all trieCoded values with higher index get the last field name.
+   * <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}.
+   * <p>This is the counterpart to {@link TrieUtils#addIndexedFields(Document,String[],String[])}.
+   */
+  public IntTrieRangeFilter(final String[] fields, final int precisionStep,
+    Integer min, Integer max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    super(fields, 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, int level) {
+            try {
+              fillBits(
+                reader, bits, termDocs,
+                fields[Math.min(fields.length-1, level)],
+                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;
+    }
+  }
+
+}

Added: 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=744207&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java (added)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/LongTrieRangeFilter.java Fri Feb 13 18:27:01 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 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 {@link TrieUtils} 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}.
+   * <p>This is the counterpart to {@link TrieUtils#addIndexedFields(Document,String,String[])}.
+   * <p><b>This is the recommended usage of TrieUtils/LongTrieRangeFilter.</b>
+   */
+  public LongTrieRangeFilter(final String field, final int precisionStep,
+    final Long min, final Long max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    this(
+      new String[]{field, field+TrieUtils.LOWER_PRECISION_FIELD_NAME_SUFFIX},
+      precisionStep,min,max,minInclusive,maxInclusive
+    );
+  }
+  
+  /**
+   * Expert: A trie filter for matching trie coded values using the given field names.
+   * You can specify the main and helper field name, that was used to idex the values.
+   * <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}.
+   * <p>This is the counterpart to {@link TrieUtils#addIndexedFields(Document,String,String,String[])}.
+   */
+  public LongTrieRangeFilter(final String field, final String lowerPrecisionField, final int precisionStep,
+    final Long min, final Long max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    this(new String[]{field, lowerPrecisionField},precisionStep,min,max,minInclusive,maxInclusive);
+  }
+
+  /**
+   * Expert: A trie filter for matching trie coded values
+   * using the given field names. If the array of field names is shorter than the
+   * trieCoded one, all trieCoded values with higher index get the last field name.
+   * <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}.
+   * <p>This is the counterpart to {@link TrieUtils#addIndexedFields(Document,String[],String[])}.
+   */
+  public LongTrieRangeFilter(final String[] fields, final int precisionStep,
+    Long min, Long max, final boolean minInclusive, final boolean maxInclusive
+  ) {
+    super(fields, 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, int level) {
+            try {
+              fillBits(
+                reader, bits, termDocs,
+                fields[Math.min(fields.length-1, level)],
+                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;
+    }
+  }
+
+}

Modified: lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieUtils.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieUtils.java?rev=744207&r1=744206&r2=744207&view=diff
==============================================================================
--- lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieUtils.java (original)
+++ lucene/java/trunk/contrib/queries/src/java/org/apache/lucene/search/trie/TrieUtils.java Fri Feb 13 18:27:01 2009
@@ -17,385 +17,531 @@
  * limitations under the License.
  */
 
-import java.util.Date;
-
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.ExtendedFieldCache;
 
 /**
  * This is a helper class to construct the trie-based index entries for numerical values.
- * <p>For more information on how the algorithm works, see the package description {@link org.apache.lucene.search.trie}.
- * The format of how the numerical values are stored in index is documented here:
- * <p>All numerical values are first converted to special <code>unsigned long</code>s by applying some bit-wise transformations. This means:<ul>
- * <li>{@link Date}s are casted to UNIX timestamps (milliseconds since 1970-01-01, this is how Java represents date/time
- * internally): {@link Date#getTime()}. The resulting <code>signed long</code> is transformed to the unsigned form like so:</li>
- * <li><code>signed long</code>s are shifted, so that {@link Long#MIN_VALUE} is mapped to <code>0x0000000000000000</code>,
- * {@link Long#MAX_VALUE} is mapped to <code>0xffffffffffffffff</code>.</li>
- * <li><code>double</code>s are converted by getting their IEEE 754 floating-point "double format" bit layout and then some bits
- * are swapped, to be able to compare the result as <code>unsigned long</code>s.</li>
- * </ul>
- * <p>For each variant (you can choose between {@link #VARIANT_8BIT}, {@link #VARIANT_4BIT}, and {@link #VARIANT_2BIT}),
- * the bitmap of this <code>unsigned long</code> is divided into parts of a number of bits (starting with the most-significant bits)
- * and each part converted to characters between {@link #TRIE_CODED_SYMBOL_MIN} and {@link #TRIE_CODED_SYMBOL_MAX}.
- * The resulting {@link String} is comparable like the corresponding <code>unsigned long</code>.
- * <p>To store the different precisions of the long values (from one character [only the most significant one] to the full encoded length),
- * each lower precision is prefixed by the length ({@link #TRIE_CODED_PADDING_START}<code>+precision == 0x20+precision</code>),
- * in an extra "helper" field with a suffixed field name (i.e. fieldname "numeric" =&gt; lower precision's name "numeric#trie").
- * The full long is not prefixed at all and indexed and stored according to the given flags in the original field name.
- * By this it is possible to get the correct enumeration of terms in correct precision
- * of the term list by just jumping to the correct fieldname and/or prefix. The full precision value may also be
- * stored in the document. Having the full precision value as term in a separate field with the original name,
- * sorting of query results against such fields is possible using the original field name.
+ * For more information on how the algorithm works, see the
+ * {@linkplain org.apache.lucene.search.trie package description}.
+ * <h3>The trie format using prefix encoded numerical values</h3>
+ * <p>To quickly execute range queries in Apache Lucene, a range is divided recursively
+ * into multiple intervals for searching: The center of the range is searched only with
+ * the lowest possible precision in the trie, while the boundaries are matched
+ * more exactly. This reduces the number of terms dramatically.
+ * <p>This class generates terms to achive this: First the numerical integer values need to
+ * be converted to strings. For that integer values (32 bit or 64 bit) are made unsigned
+ * and the bits are converted to ASCII chars with each 7 bit. The resulting string is
+ * sortable like the original integer value.
+ * <p>To also index floating point numbers, this class supplies two methods to convert them
+ * to integer values by changing their bit layout: {@link #doubleToSortableLong},
+ * {@link #floatToSortableInt}. You will have no precision loss by
+ * converting floating point numbers to integers and back (only that the integer form does
+ * is not usable). Other data types like dates can easily converted to longs or ints (e.g.
+ * date to long: {@link java.util.Date#getTime}).
+ * <p>To index the different precisions of the long values each encoded value is also 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. This series of
+ * different precision values can be indexed into a Lucene {@link Document} using
+ * {@link #addIndexedFields(Document,String,String[])}. The default is to index the original
+ * precision in the supplied field name and the lower precisions in an additional helper field.
+ * Because of this, the full-precision field can also be sorted (using {@link #getLongSortField}
+ * or {@link #getIntSortField}).
+ * <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}.
  */
 public final class TrieUtils {
 
-  /** Instance of TrieUtils using a trie factor of 8 bit.
-   * This is the <b>recommended</b> one (rather fast and storage optimized) */
-  public static final TrieUtils VARIANT_8BIT=new TrieUtils(8);
-
-  /** Instance of TrieUtils using a trie factor of 4 bit. */
-  public static final TrieUtils VARIANT_4BIT=new TrieUtils(4);
-
-  /** Instance of TrieUtils using a trie factor of 2 bit.
-   * This may be good for some indexes, but it needs much storage space
-   * and is not much faster than 8 bit in most cases. */
-  public static final TrieUtils VARIANT_2BIT=new TrieUtils(2);
+  private TrieUtils() {} // no instance!
 
-  /** Marker (PADDING)  before lower-precision trie entries to signal the precision value. See class description! */
-  public static final char TRIE_CODED_PADDING_START=(char)0x20;
-  
-  /** The "helper" field containing the lower precision terms is the original fieldname with this appended. */
+  /**
+   * The default &quot;helper&quot; field containing the lower precision terms is the original
+   * fieldname with this appended. This suffix is used in
+   * {@link #addIndexedFields(Document,String,String[])} and the corresponding c'tor
+   * of <code>(Long|Int)TrieRangeFilter</code>.
+   */
   public static final String LOWER_PRECISION_FIELD_NAME_SUFFIX="#trie";
 
-  /** Character used as lower end */
-  public static final char TRIE_CODED_SYMBOL_MIN=(char)0x100;
+  /**
+   * Longs are stored at lower precision by shifting off lower bits. The shift count is
+   * stored as <code>SHIFT_START_LONG+shift</code> in the first character
+   */
+  public static final char SHIFT_START_LONG = (char)0x20;
 
   /**
-   * A parser instance for filling a {@link ExtendedFieldCache}, that parses trie encoded fields as longs,
-   * auto detecting the trie encoding variant using the String length.
+   * Integers are stored at lower precision by shifting off lower bits. The shift count is
+   * stored as <code>SHIFT_START_INT+shift</code> in the first character
    */
-  public static final ExtendedFieldCache.LongParser FIELD_CACHE_LONG_PARSER_AUTO=new ExtendedFieldCache.LongParser(){
-    public final long parseLong(String val) {
-      return trieCodedToLongAuto(val);
-    }
-  };
-  
+  public static final char SHIFT_START_INT  = (char)0x60;
+
   /**
-   * A parser instance for filling a {@link ExtendedFieldCache}, that parses trie encoded fields as doubles,
-   * auto detecting the trie encoding variant using the String length.
+   * A parser instance for filling a {@link ExtendedFieldCache}, that parses prefix encoded fields as longs.
    */
-  public static final ExtendedFieldCache.DoubleParser FIELD_CACHE_DOUBLE_PARSER_AUTO=new ExtendedFieldCache.DoubleParser(){
-    public final double parseDouble(String val) {
-      return trieCodedToDoubleAuto(val);
+  public static final ExtendedFieldCache.LongParser FIELD_CACHE_LONG_PARSER=new ExtendedFieldCache.LongParser(){
+    public final long parseLong(final String val) {
+      return prefixCodedToLong(val);
     }
   };
   
   /**
-   * Detects and returns the variant of a trie encoded string using the length.
-   * @throws NumberFormatException if the length is not 8, 16, or 32 chars.
+   * A parser instance for filling a {@link FieldCache}, that parses prefix encoded fields as ints.
    */
-  public static final TrieUtils autoDetectVariant(final String s) {
-    final int l=s.length();
-    if (l==VARIANT_8BIT.TRIE_CODED_LENGTH) {
-      return VARIANT_8BIT;
-    } else if (l==VARIANT_4BIT.TRIE_CODED_LENGTH) {
-      return VARIANT_4BIT;
-    } else if (l==VARIANT_2BIT.TRIE_CODED_LENGTH) {
-      return VARIANT_2BIT;
-    } else {
-      throw new NumberFormatException("Invalid trie encoded numerical value representation (incompatible length).");
+  public static final FieldCache.IntParser FIELD_CACHE_INT_PARSER=new FieldCache.IntParser(){
+    public final int parseInt(final String val) {
+      return prefixCodedToInt(val);
     }
-  }
-
-  /**
-   * Converts a encoded <code>String</code> value back to a <code>long</code>,
-   * auto detecting the trie encoding variant using the String length.
-   */
-  public static final long trieCodedToLongAuto(final String s) {
-    return autoDetectVariant(s).trieCodedToLong(s);
-  }
-
-  /**
-   * Converts a encoded <code>String</code> value back to a <code>double</code>,
-   * auto detecting the trie encoding variant using the String length.
-   */
-  public static final double trieCodedToDoubleAuto(final String s) {
-    return autoDetectVariant(s).trieCodedToDouble(s);
-  }
-
-  /**
-   * Converts a encoded <code>String</code> value back to a <code>Date</code>,
-   * auto detecting the trie encoding variant using the String length.
-   */
-  public static final Date trieCodedToDateAuto(final String s) {
-    return autoDetectVariant(s).trieCodedToDate(s);
-  }
+  };
 
   /**
-   * A factory method, that generates a {@link SortField} instance for sorting trie encoded values,
-   * automatically detecting the trie encoding variant using the String length.
+   * A parser instance for filling a {@link ExtendedFieldCache}, that parses prefix encoded fields as doubles.
+   * This uses {@link #sortableLongToDouble} to convert the encoded long to a double.
    */
-  public static final SortField getSortFieldAuto(final String field) {
-    return new SortField(field, FIELD_CACHE_LONG_PARSER_AUTO);
-  }
+  public static final ExtendedFieldCache.DoubleParser FIELD_CACHE_DOUBLE_PARSER=new ExtendedFieldCache.DoubleParser(){
+    public final double parseDouble(final String val) {
+      return sortableLongToDouble(prefixCodedToLong(val));
+    }
+  };
   
   /**
-   * A factory method, that generates a {@link SortField} instance for sorting trie encoded values,
-   * automatically detecting the trie encoding variant using the String length.
+   * A parser instance for filling a {@link FieldCache}, that parses prefix encoded fields as floats.
+   * This uses {@link #sortableIntToFloat} to convert the encoded int to a float.
    */
-  public static final SortField getSortFieldAuto(final String field, boolean reverse) {
-    return new SortField(field, FIELD_CACHE_LONG_PARSER_AUTO, reverse);
-  }
-  
-  // TrieUtils instance's part
-  
-  private TrieUtils(int bits) {
-    assert 64%bits == 0;
-    
-    // helper variable for conversion
-    mask = (1L << bits) - 1L;		
-
-    // init global "constants"
-    TRIE_BITS=bits;
-    TRIE_CODED_LENGTH=64/TRIE_BITS;
-    TRIE_CODED_SYMBOL_MAX=(char)(TRIE_CODED_SYMBOL_MIN+mask);
-    TRIE_CODED_NUMERIC_MIN=longToTrieCoded(Long.MIN_VALUE);
-    TRIE_CODED_NUMERIC_MAX=longToTrieCoded(Long.MAX_VALUE);
-  }
-
-  // internal conversion to/from strings
-
-  private final String internalLongToTrieCoded(long l) {
-    final char[] buf=new char[TRIE_CODED_LENGTH];
-    for (int i=TRIE_CODED_LENGTH-1; i>=0; i--) {
-      buf[i] = (char)( TRIE_CODED_SYMBOL_MIN + (l & mask) );
-      l = l >>> TRIE_BITS;
+  public static final FieldCache.FloatParser FIELD_CACHE_FLOAT_PARSER=new FieldCache.FloatParser(){
+    public final float parseFloat(final String val) {
+      return sortableIntToFloat(prefixCodedToInt(val));
     }
-    return new String(buf);
-  }
+  };
 
-  private final long internalTrieCodedToLong(final String s) {
-    if (s==null) throw new NullPointerException("Trie encoded string may not be NULL");
-    final int len=s.length();
-    if (len!=TRIE_CODED_LENGTH) throw new NumberFormatException(
-      "Invalid trie encoded numerical value representation (incompatible length, must be "+TRIE_CODED_LENGTH+")"
-    );
-    long l=0L;
-    for (int i=0; i<len; i++) {
-      char ch=s.charAt(i);
-      if (ch>=TRIE_CODED_SYMBOL_MIN && ch<=TRIE_CODED_SYMBOL_MAX) {
-        l = (l << TRIE_BITS) | (long)(ch-TRIE_CODED_SYMBOL_MIN);
-      } else {
+  /**
+   * This is a convenience method, that returns prefix coded bits of a long without
+   * reducing the precision. It can be used to store the full precision value as a
+   * stored field in index.
+   * <p>To decode, use {@link #prefixCodedToLong}.
+   */
+  public static String longToPrefixCoded(final long val) {
+    return longToPrefixCoded(val, 0);
+  }
+
+  /**
+   * Expert: Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
+   * This is method is used by {@link #trieCodeLong}.
+   */
+  public static String longToPrefixCoded(final long val, final int shift) {
+    if (shift>63 || shift<0)
+      throw new IllegalArgumentException("Illegal shift value, must be 0..63");
+    int nChars = (63-shift)/7 + 1;
+    final char[] arr = new char[nChars+1];
+    arr[0] = (char)(SHIFT_START_LONG + shift);
+    long sortableBits = val ^ 0x8000000000000000L;
+    sortableBits >>>= shift;
+    while (nChars>=1) {
+      // Store 7 bits per character for good efficiency when UTF-8 encoding.
+      // The whole number is right-justified so that lucene can prefix-encode
+      // the terms more efficiently.
+      arr[nChars--] = (char)(sortableBits & 0x7f);
+      sortableBits >>>= 7;
+    }
+    return new String(arr);
+  }
+
+  /**
+   * This is a convenience method, that returns prefix coded bits of an int without
+   * reducing the precision. It can be used to store the full precision value as a
+   * stored field in index.
+   * <p>To decode, use {@link #prefixCodedToInt}.
+   */
+  public static String intToPrefixCoded(final int val) {
+    return intToPrefixCoded(val, 0);
+  }
+
+  /**
+   * Expert: Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
+   * This is method is used by {@link #trieCodeInt}.
+   */
+  public static String intToPrefixCoded(final int val, final int shift) {
+    if (shift>31 || shift<0)
+      throw new IllegalArgumentException("Illegal shift value, must be 0..31");
+    int nChars = (31-shift)/7 + 1;
+    final char[] arr = new char[nChars+1];
+    arr[0] = (char)(SHIFT_START_INT + shift);
+    int sortableBits = val ^ 0x80000000;
+    sortableBits >>>= shift;
+    while (nChars>=1) {
+      // Store 7 bits per character for good efficiency when UTF-8 encoding.
+      // The whole number is right-justified so that lucene can prefix-encode
+      // the terms more efficiently.
+      arr[nChars--] = (char)(sortableBits & 0x7f);
+      sortableBits >>>= 7;
+    }
+    return new String(arr);
+  }
+
+  /**
+   * Returns a long from prefixCoded characters.
+   * Rightmost bits will be zero for lower precision codes.
+   * This method can be used to decode e.g. a stored field.
+   * @see #longToPrefixCoded(long)
+   */
+  public static long prefixCodedToLong(final String prefixCoded) {
+    final int len = prefixCoded.length();
+    final int shift = prefixCoded.charAt(0)-SHIFT_START_LONG;
+    if (shift>63 || shift<0)
+      throw new NumberFormatException("Invalid shift value in prefixCoded string (is encoded value really a LONG?)");
+    long sortableBits = 0L;
+    for (int i=1; i<len; i++) {
+      sortableBits <<= 7;
+      final char ch = prefixCoded.charAt(i);
+      if (ch>0x7f) {
         throw new NumberFormatException(
-          "Invalid trie encoded numerical value representation (char "+
+          "Invalid prefixCoded numerical value representation (char "+
           Integer.toHexString((int)ch)+" at position "+i+" is invalid)"
         );
       }
+      sortableBits |= (long)(ch & 0x7f);
     }
-    return l;
+    return (sortableBits << shift) ^ 0x8000000000000000L;
   }
 
-  // Long's
-
-  /** Converts a <code>long</code> value encoded to a <code>String</code>. */
-  public String longToTrieCoded(final long l) {
-    return internalLongToTrieCoded(l ^ 0x8000000000000000L);
-  }
-
-  /** Converts a encoded <code>String</code> value back to a <code>long</code>. */
-  public long trieCodedToLong(final String s) {
-    return internalTrieCodedToLong(s) ^ 0x8000000000000000L;
-  }
-
-  // Double's
-
-  /** Converts a <code>double</code> value encoded to a <code>String</code>. */
-  public String doubleToTrieCoded(final double d) {
-    long l=Double.doubleToLongBits(d);
-    if ((l & 0x8000000000000000L) == 0L) {
-      // >0
-      l |= 0x8000000000000000L;
-    } else {
-      // <0
-      l = ~l;
+  /**
+   * Returns an int from prefixCoded characters.
+   * Rightmost bits will be zero for lower precision codes.
+   * This method can be used to decode e.g. a stored field.
+   * @see #intToPrefixCoded(int)
+   */
+  public static int prefixCodedToInt(final String prefixCoded) {
+    final int len = prefixCoded.length();
+    final int shift = prefixCoded.charAt(0)-SHIFT_START_INT;
+    if (shift>31 || shift<0)
+      throw new NumberFormatException("Invalid shift value in prefixCoded string (is encoded value really an INT?)");
+    int sortableBits = 0;
+    for (int i=1; i<len; i++) {
+      sortableBits <<= 7;
+      final char ch = prefixCoded.charAt(i);
+      if (ch>0x7f) {
+        throw new NumberFormatException(
+          "Invalid prefixCoded numerical value representation (char "+
+          Integer.toHexString((int)ch)+" at position "+i+" is invalid)"
+        );
+      }
+      sortableBits |= (int)(ch & 0x7f);
     }
-    return internalLongToTrieCoded(l);
+    return (sortableBits << shift) ^ 0x80000000;
   }
 
-  /** Converts a encoded <code>String</code> value back to a <code>double</code>. */
-  public double trieCodedToDouble(final String s) {
-    long l=internalTrieCodedToLong(s);
-    if ((l & 0x8000000000000000L) != 0L) {
-      // >0
-      l &= 0x7fffffffffffffffL;
-    } else {
-      // <0
-      l = ~l;
-    }
-    return Double.longBitsToDouble(l);
+  /**
+   * Converts a <code>double</code> value to a sortable signed <code>long</code>.
+   * The value is converted by getting their IEEE 754 floating-point &quot;double format&quot;
+   * bit layout and then some bits are swapped, to be able to compare the result as long.
+   * By this the precision is not reduced, but the value can easily used as a long.
+   * @see #sortableLongToDouble
+   */
+  public static long doubleToSortableLong(double val) {
+    long f = Double.doubleToLongBits(val);
+    if (f<0) f ^= 0x7fffffffffffffffL;
+    return f;
   }
 
-  // Date's
-
-  /** Converts a <code>Date</code> value encoded to a <code>String</code>. */
-  public String dateToTrieCoded(final Date d) {
-    return longToTrieCoded(d.getTime());
+  /**
+   * Converts a sortable <code>long</code> back to a <code>double</code>.
+   * @see #doubleToSortableLong
+   */
+  public static double sortableLongToDouble(long val) {
+    if (val<0) val ^= 0x7fffffffffffffffL;
+    return Double.longBitsToDouble(val);
   }
 
-  /** Converts a encoded <code>String</code> value back to a <code>Date</code>. */
-  public Date trieCodedToDate(final String s) {
-    return new Date(trieCodedToLong(s));
+  /**
+   * Converts a <code>float</code> value to a sortable signed <code>int</code>.
+   * The value is converted by getting their IEEE 754 floating-point &quot;float format&quot;
+   * bit layout and then some bits are swapped, to be able to compare the result as int.
+   * By this the precision is not reduced, but the value can easily used as an int.
+   * @see #sortableIntToFloat
+   */
+  public static int floatToSortableInt(float val) {
+    int f = Float.floatToIntBits(val);
+    if (f<0) f ^= 0x7fffffff;
+    return f;
   }
 
-  // increment / decrement
-
-  /** Increments an encoded String value by 1. Needed by {@link TrieRangeFilter}. */
-  public String incrementTrieCoded(final String v) {
-    final int l=v.length();
-    final char[] buf=new char[l];
-    boolean inc=true;
-    for (int i=l-1; i>=0; i--) {
-      int b=v.charAt(i)-TRIE_CODED_SYMBOL_MIN;
-      if (inc) b++;
-      if (inc=(b>(int)mask)) b=0;
-      buf[i]=(char)(TRIE_CODED_SYMBOL_MIN+b);
-    }
-    return new String(buf);
+  /**
+   * Converts a sortable <code>int</code> back to a <code>float</code>.
+   * @see #floatToSortableInt
+   */
+  public static float sortableIntToFloat(int val) {
+    if (val<0) val ^= 0x7fffffff;
+    return Float.intBitsToFloat(val);
   }
 
-  /** Decrements an encoded String value by 1. Needed by {@link TrieRangeFilter}. */
-  public String decrementTrieCoded(final String v) {
-    final int l=v.length();
-    final char[] buf=new char[l];
-    boolean dec=true;
-    for (int i=l-1; i>=0; i--) {
-      int b=v.charAt(i)-TRIE_CODED_SYMBOL_MIN;
-      if (dec) b--;
-      if (dec=(b<0)) b=(int)mask;
-      buf[i]=(char)(TRIE_CODED_SYMBOL_MIN+b);
-    }
-    return new String(buf);
+  /** A factory method, that generates a {@link SortField} instance for sorting prefix encoded long values. */
+  public static SortField getLongSortField(final String field, boolean reverse) {
+    return new SortField(field, FIELD_CACHE_LONG_PARSER, reverse);
   }
-
-  private void addConvertedTrieCodedDocumentField(
-    final Document ldoc, final String fieldname, final String val,
-    final boolean index, final Field.Store store
-  ) {
-    Field f=new Field(fieldname, val, store, index?Field.Index.NOT_ANALYZED_NO_NORMS:Field.Index.NO);
-    if (index) {
+  
+  /** A factory method, that generates a {@link SortField} instance for sorting prefix encoded int values. */
+  public static SortField getIntSortField(final String field, boolean reverse) {
+    return new SortField(field, FIELD_CACHE_INT_PARSER, reverse);
+  }
+
+  /**
+   * Returns a sequence of trie coded numbers suitable for {@link LongTrieRangeFilter}.
+   * Each successive string in the list has had it's precision reduced by <code>precisionStep</code>.
+   * For sorting, index the first full-precision value into a separate field and the
+   * remaining values into another field.
+   * <p>To achieve this, use {@link #addIndexedFields(Document,String,String[])}.
+   */
+  public static String[] trieCodeLong(long val, int precisionStep) {
+    if (precisionStep<1 || precisionStep>64)
+      throw new IllegalArgumentException("precisionStep may only be 1..64");
+    String[] arr = new String[63/precisionStep+1];
+    int idx = 0;
+    for (int shift=0; shift<64; shift+=precisionStep) {
+      arr[idx++] = longToPrefixCoded(val, shift);
+    }
+    return arr;
+  }
+
+  /**
+   * Returns a sequence of trie coded numbers suitable for {@link IntTrieRangeFilter}.
+   * Each successive string in the list has had it's precision reduced by <code>precisionStep</code>.
+   * For sorting, index the first full-precision value into a separate field and the
+   * remaining values into another field.
+   * <p>To achieve this, use {@link #addIndexedFields(Document,String,String[])}.
+   */
+  public static String[] trieCodeInt(int val, int precisionStep) {
+    if (precisionStep<1 || precisionStep>32)
+      throw new IllegalArgumentException("precisionStep may only be 1..32");
+    String[] arr = new String[31/precisionStep+1];
+    int idx = 0;
+    for (int shift=0; shift<32; shift+=precisionStep) {
+      arr[idx++] = intToPrefixCoded(val, shift);
+    }
+    return arr;
+  }
+
+  /**
+   * Indexes the full precision value only in the main field (for sorting), and indexes all other
+   * lower precision values in <code>field+LOWER_PRECISION_FIELD_NAME_SUFFIX</code>.
+   * <p><b>This is the recommended variant to add trie fields to the index.</b>
+   * By this it is possible to sort the field using a <code>SortField</code> instance
+   * returned by {@link #getLongSortField} or {@link #getIntSortField}.
+   * <p>This method does not store the fields and saves no term frequency or norms
+   * (which are normally not needed for trie fields). If you want to additionally store
+   * the value, you can use the normal methods of {@link Document} to achive this, just specify
+   * <code>Field.Store.YES</code>, <code>Field.Index.NO</code> and the same field name.
+   * <p>Examples:
+   * <pre>
+   *  addIndexedFields(doc, "mydouble", trieCodeLong(doubleToSortableLong(1.414d), 4));
+   *  addIndexedFields(doc, "mylong", trieCodeLong(123456L, 4));
+   * </pre>
+   **/
+  public static void addIndexedFields(Document doc, String field, String[] trieCoded) {
+    addIndexedFields(doc, new String[]{field, field+LOWER_PRECISION_FIELD_NAME_SUFFIX}, trieCoded);
+  }
+
+  /**
+   * Expert: Indexes the full precision value only in the main field (for sorting), and indexes all other
+   * lower precision values in the <code>lowerPrecision</code> field.
+   * If you do not specify the same field name for the main and lower precision one,
+   * it is possible to sort the field using a <code>SortField</code> instance
+   * returned by {@link #getLongSortField} or {@link #getIntSortField}.
+   * <p>This method does not store the fields and saves no term frequency or norms
+   * (which are normally not needed for trie fields). If you want to additionally store
+   * the value, you can use the normal methods of {@link Document} to achive this, just specify
+   * <code>Field.Store.YES</code>, <code>Field.Index.NO</code> and the same main field name.
+   * <p>Examples:
+   * <pre>
+   *  addIndexedFields(doc, "mydouble", "mydoubletrie", trieCodeLong(doubleToSortableLong(1.414d), 4));
+   *  addIndexedFields(doc, "mylong", "mylongtrie", trieCodeLong(123456L, 4));
+   * </pre>
+   * @see #addIndexedFields(Document,String,String[])
+   **/
+  public static void addIndexedFields(Document doc, String field, String lowerPrecisionField, String[] trieCoded) {
+    addIndexedFields(doc, new String[]{field, lowerPrecisionField}, trieCoded);
+  }
+
+  /**
+   * Expert: Indexes a series of trie coded values into a lucene {@link Document}
+   * using the given field names.
+   * If the array of field names is shorter than the trie coded one, all trie coded
+   * values with higher index get the last field name.
+   * <p>This method does not store the fields and saves no term frequency or norms
+   * (which are normally not needed for trie fields). If you want to additionally store
+   * the value, you can use the normal methods of {@link Document} to achive this, just specify
+   * <code>Field.Store.YES</code>, <code>Field.Index.NO</code> and the same main field name.
+   **/
+  public static void addIndexedFields(Document doc, String[] fields, String[] trieCoded) {
+    for (int i=0; i<trieCoded.length; i++) {
+      final int fnum = Math.min(fields.length-1, i);
+      final Field f = new Field(fields[fnum], trieCoded[i], Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS);
       f.setOmitTf(true);
-      ldoc.add(f);
-      // add the lower precision values in the helper field with prefix
-      final StringBuffer sb=new StringBuffer(TRIE_CODED_LENGTH);
-      synchronized(sb) {
-        for (int i=TRIE_CODED_LENGTH-1; i>0; i--) {
-          sb.setLength(0);
-          f=new Field(
-            fieldname + LOWER_PRECISION_FIELD_NAME_SUFFIX,
-            sb.append( (char)(TRIE_CODED_PADDING_START+i) ).append( val.substring(0,i) ).toString(),
-            Field.Store.NO, Field.Index.NOT_ANALYZED_NO_NORMS
-          );
-          f.setOmitTf(true);
-          ldoc.add(f);
-        }
-      }
-    } else {
-      ldoc.add(f);
+      doc.add(f);
     }
   }
 
   /**
-   * Stores a double value in trie-form in document for indexing.
-   * <p>To store the different precisions of the long values (from one byte [only the most significant one] to the full eight bytes),
-   * each lower precision is prefixed by the length ({@link #TRIE_CODED_PADDING_START}<code>+precision</code>),
-   * in an extra "helper" field with a name of <code>fieldname+{@link #LOWER_PRECISION_FIELD_NAME_SUFFIX}</code>
-   * (i.e. fieldname "numeric" => lower precision's name "numeric#trie").
-   * The full long is not prefixed at all and indexed and stored according to the given flags in the original field name.
-   * If the field should not be searchable, set <code>index</code> to <code>false</code>. It is then only stored (for convenience).
-   * Fields added to a document using this method can be queried by {@link TrieRangeQuery}. 
+   * Expert: Splits a long range recursively.
+   * You may implement a builder that adds clauses to a
+   * {@link org.apache.lucene.search.BooleanQuery} for each call to its
+   * {@link IntRangeBuilder#addRange(String,String,int)}
+   * method.
+   * <p>This method is used by {@link LongTrieRangeFilter}.
    */
-  public void addDoubleTrieCodedDocumentField(
-    final Document ldoc, final String fieldname, final double val,
-    final boolean index, final Field.Store store
+  public static void splitLongRange(final LongRangeBuilder builder,
+    final int precisionStep,  final long minBound, final long maxBound
   ) {
-    addConvertedTrieCodedDocumentField(ldoc, fieldname, doubleToTrieCoded(val), index, store);
+    if (precisionStep<1 || precisionStep>64)
+      throw new IllegalArgumentException("precisionStep may only be 1..64");
+    splitRange(
+      builder, 64, precisionStep, minBound, maxBound,
+      0 /* start with no shift */
+    );
   }
-
+  
   /**
-   * Stores a Date value in trie-form in document for indexing.
-   * <p>To store the different precisions of the long values (from one byte [only the most significant one] to the full eight bytes),
-   * each lower precision is prefixed by the length ({@link #TRIE_CODED_PADDING_START}<code>+precision</code>),
-   * in an extra "helper" field with a name of <code>fieldname+{@link #LOWER_PRECISION_FIELD_NAME_SUFFIX}</code>
-   * (i.e. fieldname "numeric" => lower precision's name "numeric#trie").
-   * The full long is not prefixed at all and indexed and stored according to the given flags in the original field name.
-   * If the field should not be searchable, set <code>index</code> to <code>false</code>. It is then only stored (for convenience).
-   * Fields added to a document using this method can be queried by {@link TrieRangeQuery}. 
-   */
-  public void addDateTrieCodedDocumentField(
-    final Document ldoc, final String fieldname,
-    final Date val, final boolean index, final Field.Store store
+   * Expert: Splits an int range recursively.
+   * You may implement a builder that adds clauses to a
+   * {@link org.apache.lucene.search.BooleanQuery} for each call to its
+   * {@link IntRangeBuilder#addRange(String,String,int)}
+   * method.
+   * <p>This method is used by {@link IntTrieRangeFilter}.
+   */
+  public static void splitIntRange(final IntRangeBuilder builder,
+    final int precisionStep,  final int minBound, final int maxBound
   ) {
-    addConvertedTrieCodedDocumentField(ldoc, fieldname, dateToTrieCoded(val), index, store);
+    if (precisionStep<1 || precisionStep>32)
+      throw new IllegalArgumentException("precisionStep may only be 1..32");
+    splitRange(
+      builder, 32, precisionStep, (long)minBound, (long)maxBound,
+      0 /* start with no shift */
+    );
   }
-
-  /**
-   * Stores a long value in trie-form in document for indexing.
-   * <p>To store the different precisions of the long values (from one byte [only the most significant one] to the full eight bytes),
-   * each lower precision is prefixed by the length ({@link #TRIE_CODED_PADDING_START}<code>+precision</code>),
-   * in an extra "helper" field with a name of <code>fieldname+{@link #LOWER_PRECISION_FIELD_NAME_SUFFIX}</code>
-   * (i.e. fieldname "numeric" => lower precision's name "numeric#trie").
-   * The full long is not prefixed at all and indexed and stored according to the given flags in the original field name.
-   * If the field should not be searchable, set <code>index</code> to <code>false</code>. It is then only stored (for convenience).
-   * Fields added to a document using this method can be queried by {@link TrieRangeQuery}. 
-   */
-  public void addLongTrieCodedDocumentField(
-    final Document ldoc, final String fieldname,
-    final long val, final boolean index, final Field.Store store
+  
+  /** This helper does the splitting for both 32 and 64 bit. */
+  private static void splitRange(
+    final Object builder, final int valSize,
+    final int precisionStep, final long minBound, final long maxBound,
+    final int shift
   ) {
-    addConvertedTrieCodedDocumentField(ldoc, fieldname, longToTrieCoded(val), index, store);
+    // calculate new bounds for inner precision
+    final long diff = 1L << (shift+precisionStep),
+      mask = ((1L<<precisionStep) - 1L) << shift;
+    final boolean
+      hasLower = (minBound & mask) != 0L,
+      hasUpper = (maxBound & mask) != mask;
+    final long
+      nextMinBound = (hasLower ? (minBound + diff) : minBound) & ~mask,
+      nextMaxBound = (hasUpper ? (maxBound - diff) : maxBound) & ~mask;
+
+    if (shift+precisionStep>=valSize || nextMinBound>nextMaxBound) {
+      // We are in the lowest precision or the next precision is not available.
+      addRange(builder, valSize, precisionStep, minBound, maxBound, shift);
+    } else {
+      if (hasLower)
+        addRange(builder, valSize, precisionStep, minBound, minBound | mask, shift);
+      if (hasUpper)
+        addRange(builder, valSize, precisionStep, maxBound & ~mask, maxBound, shift);
+      // recurse down to next precision
+      splitRange(
+        builder, valSize, precisionStep,
+        nextMinBound, nextMaxBound,
+        shift+precisionStep
+      );
+    }
   }
   
-  /** A factory method, that generates a {@link SortField} instance for sorting trie encoded values. */
-  public SortField getSortField(final String field) {
-    return new SortField(field, FIELD_CACHE_LONG_PARSER);
+  /** Helper that delegates to correct range builder */
+  private static void addRange(
+    final Object builder, final int valSize,
+    final int precisionStep, long minBound, long maxBound,
+    final int shift
+  ) {
+    // for the max bound set all lower bits (that were shifted away):
+    // this is important for testing or other usages of the splitted range
+    // (e.g. to reconstruct the full range). The prefixEncoding will remove
+    // the bits anyway, so they do not hurt!
+    maxBound |= (1L << shift) - 1L;
+    // delegate to correct range builder
+    switch(valSize) {
+      case 64:
+        ((LongRangeBuilder)builder).addRange(precisionStep, minBound, maxBound, shift);
+        break;
+      case 32:
+        ((IntRangeBuilder)builder).addRange(precisionStep, (int)minBound, (int)maxBound, shift);
+        break;
+      default:
+        // Should not happen!
+        throw new IllegalArgumentException("valSize must be 32 or 64.");
+    }
   }
+
+  /**
+   * Expert: Callback for {@link #splitLongRange}.
+   * You need to overwrite only one of the methods.
+   */
+  public static abstract class LongRangeBuilder {
+    
+    /**
+     * Overwrite this method, if you like to receive the already prefix encoded range bounds.
+     * You can directly build classical range queries from them.
+     * The level gives the precision level (0 = highest precision) of the encoded values.
+     * This parameter could be used as an index to an array of fieldnames like the
+     * parameters to {@link #addIndexedFields(Document,String[],String[])} for specifying
+     * the field names for each precision:
+     * <pre>
+     *  String field = fields[Math.min(fields.length-1, level)];
+     * </pre>
+     */
+    public void addRange(String minPrefixCoded, String maxPrefixCoded, int level) {
+      throw new UnsupportedOperationException();
+    }
+    
+    /**
+     * Overwrite this method, if you like to receive the raw long range bounds.
+     * You can use this for e.g. debugging purposes (print out range bounds).
+     */
+    public void addRange(final int precisionStep, final long min, final long max, final int shift) {
+      /*System.out.println(Long.toHexString((min^0x8000000000000000L) >>> shift)+".."+
+        Long.toHexString((max^0x8000000000000000L) >>> shift));*/
+      addRange(longToPrefixCoded(min, shift), longToPrefixCoded(max, shift), shift/precisionStep);
+    }
   
-  /** A factory method, that generates a {@link SortField} instance for sorting trie encoded values. */
-  public SortField getSortField(final String field, boolean reverse) {
-    return new SortField(field, FIELD_CACHE_LONG_PARSER, reverse);
   }
   
-  /** A parser instance for filling a {@link ExtendedFieldCache}, that parses trie encoded fields as longs. */
-  public final ExtendedFieldCache.LongParser FIELD_CACHE_LONG_PARSER=new ExtendedFieldCache.LongParser(){
-    public final long parseLong(String val) {
-      return trieCodedToLong(val);
+  /**
+   * Expert: Callback for {@link #splitIntRange}.
+   * You need to overwrite only one of the methods.
+   */
+  public static abstract class IntRangeBuilder {
+    
+    /**
+     * Overwrite this method, if you like to receive the already prefix encoded range bounds.
+     * You can directly build classical range queries from them.
+     * The level gives the precision level (0 = highest precision) of the encoded values.
+     * This parameter could be used as an index to an array of fieldnames like the
+     * parameters to {@link #addIndexedFields(Document,String[],String[])} for specifying
+     * the field names for each precision:
+     * <pre>
+     *  String field = fields[Math.min(fields.length-1, level)];
+     * </pre>
+     */
+    public void addRange(String minPrefixCoded, String maxPrefixCoded, int level) {
+      throw new UnsupportedOperationException();
     }
-  };
-  
-  /** A parser instance for filling a {@link ExtendedFieldCache}, that parses trie encoded fields as doubles. */
-  public final ExtendedFieldCache.DoubleParser FIELD_CACHE_DOUBLE_PARSER=new ExtendedFieldCache.DoubleParser(){
-    public final double parseDouble(String val) {
-      return trieCodedToDouble(val);
+    
+    /**
+     * Overwrite this method, if you like to receive the raw int range bounds.
+     * You can use this for e.g. debugging purposes (print out range bounds).
+     */
+    public void addRange(final int precisionStep, final int min, final int max, final int shift) {
+      /*System.out.println(Integer.toHexString((min^0x80000000) >>> shift)+".."+
+        Integer.toHexString((max^0x80000000) >>> shift));*/
+      addRange(intToPrefixCoded(min, shift), intToPrefixCoded(max, shift), shift/precisionStep);
     }
-  };
   
-  private final long mask;
+  }
   
-  /** Number of bits used in this trie variant (2, 4, or 8) */
-  public final int TRIE_BITS;
-
-  /** Length (in chars) of an encoded value (8, 16, or 32 chars) */
-  public final int TRIE_CODED_LENGTH;
-
-  /** Character used as upper end (depends on trie bits, its <code>{@link #TRIE_CODED_SYMBOL_MIN}+2^{@link #TRIE_BITS}-1</code>) */
-  public final char TRIE_CODED_SYMBOL_MAX;
-
-  /** minimum encoded value of a numerical index entry: {@link Long#MIN_VALUE} */
-  public final String TRIE_CODED_NUMERIC_MIN;
-
-  /** maximum encoded value of a numerical index entry: {@link Long#MAX_VALUE} */
-  public final String TRIE_CODED_NUMERIC_MAX;
-
 }

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=744207&r1=744206&r2=744207&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 Fri Feb 13 18:27:01 2009
@@ -1,7 +1,7 @@
 <html>
 <body>
-<p>This package provides fast numeric range queries/filters on <code>long</code>, <code>double</code> or <code>Date</code>
-fields based on trie structures.</p>
+<p>This package provides fast numeric range queries/filters on <code>long</code>, <code>double</code>, <code>int</code>,
+or <code>float</code> (and other data types, that can be converted to numerical values) fields based on trie structures.</p>
 
 <h3>How it works</h3>
 <p>See the publication about <a target="_blank" href="http://www.panfmp.org">panFMP</a>, where this algorithm was described:
@@ -14,66 +14,96 @@
 it cannot handle numerical ranges (e.g., field value is inside user defined bounds, even dates are numerical values).
 We have developed an extension to Apache Lucene that stores
 the numerical values in a special string-encoded format with variable precision
-(all numerical values like doubles, longs, and timestamps are converted to lexicographic sortable string representations
-and stored with different precisions from one byte to the full 8 bytes - depending on the variant used).
-For a more detailed description of how the values are stored, see {@link org.apache.lucene.search.trie.TrieUtils}.
-A range is then divided recursively into multiple intervals for searching:
-The center of the range is searched only with the lowest possible precision in the trie, while the boundaries are matched
-more exactly. This reduces the number of terms dramatically.</p>
-
-<p>For the variant that uses a lowest precision of 1-byte the index 
-contains only a maximum of 256 distinct values in the lowest precision.
-Overall, a range could consist of a theoretical maximum of
+(all numerical values like doubles, longs, floats, and ints are converted to lexicographic sortable string representations
+and stored with different precisions. For a more detailed description of how the values are stored,
+see {@link org.apache.lucene.search.trie.TrieUtils}. A range is then divided recursively into
+multiple intervals for searching:
+The center of the range is searched only with the lowest possible precision in the trie,
+while the boundaries are matched more exactly. This reduces the number of terms dramatically.</p>
+
+<p>For the variant that stores long values in 8 different precisions (each reduced by 8 bits) that
+uses a lowest precision of 1 byte, the index contains only a maximum of 256 distinct values in the
+lowest precision. Overall, a range could consist of a theoretical maximum of
 <code>7*255*2 + 255 = 3825</code> distinct terms (when there is a term for every distinct value of an
-8-byte-number in the index and the range covers all of them; a maximum of 255 distinct values is used
+8-byte-number in the index and the range covers almost all of them; a maximum of 255 distinct values is used
 because it would always be possible to reduce the full 256 values to one term with degraded precision).
 In practise, we have seen up to 300 terms in most cases (index with 500,000 metadata records
 and a uniform value distribution).</p>
 
-<p>There are two other variants of encoding: 4bit and 2bit. Each variant stores more different precisions
-of the longs and thus needs more storage space (because it generates more and longer terms -
-4bit: two times the length and number of terms; 2bit: four times the length and number of terms).
-But on the other hand, the maximum number of distinct terms used for range queries is
-<code>15*15*2 + 15 = 465</code> for the 4bit variant, and
-<code>31*3*2 + 3 = 189</code> for the 2bit variant.</p>
+<p>You can choose any <code>precisionStep</code> when encoding integer values.
+Lower step values mean more precisions and so more terms in index (and index gets larger).
+On the other hand, the maximum number of terms to match reduces, which optimized query speed.
+The formula to calculate the maximum term count is:
+<pre>
+ n = [ (bitsPerValue/precisionStep - 1) * (2^precisionStep - 1 ) * 2 ] + (2^precisionStep - 1 )
+</pre>
+<p><em>(this formula is only correct, when <code>bitsPerValue/precisionStep</code> is an integer;
+in other cases, the value must be rounded up and the last summand must contain the modulo of the division as
+precision step)</em>.
+For longs stored using a precision step of 4, <code>n = 15*15*2 + 15 = 465</code>, and for a precision
+step of 2, <code>n = 31*3*2 + 3 = 189</code>. But the faster search speed is reduced by more seeking
+in the term enum of the index. Because of this, the ideal <code>precisionStep</code> value can only
+be found out by testing. <b>Important:</b> You can index with a lower precision step value and test search speed
+using a multiple of the original step value.</p>
 
 <p>This dramatically improves the performance of Apache Lucene with range queries, which
 are no longer dependent on the index size and the number of distinct values because there is
 an upper limit unrelated to either of these properties.</p>
 
 <h3>Usage</h3>
-<p>To use the new query types the numerical values, which may be <code>long</code>, <code>double</code> or <code>Date</code>,
-the values must be stored during indexing in a special format in the index (using {@link org.apache.lucene.search.trie.TrieUtils}).
-This can be done like this:</p>
+<p>To use the new query types the numerical values, which may be<code>long</code>, <code>double</code>, <code>int</code>,
+<code>float</code>, or <code>Date</code>, the values must be indexed in a special prefix encoded format
+(using {@link org.apache.lucene.search.trie.TrieUtils}). This can be done like this:</p>
 
 <pre>
+  <em>// chose a step value, 8 is a general good value for large indexes:</em>
+  int precisionStep = 8;
+  
   Document doc = new Document();
-  // add some standard fields:
+  
+  <em>// add some standard fields:</em>
   String svalue = "anything to index";
-  doc.add(new Field("exampleString", svalue, Field.Store.YES, Field.Index.ANALYZED) ;
-  // add some numerical fields:
-  double fvalue = 1.057E17;
-  TrieUtils.VARIANT_8BIT.addDoubleTrieCodedDocumentField(doc, "exampleDouble", fvalue, true /* index the field */, Field.Store.YES);
+  doc.add(new Field("exampleString", svalue, Field.Store.YES, Field.Index.ANALYZED));
+  
+  <em>// add some numerical fields:</em>
   long lvalue = 121345L;
-  TrieUtils.VARIANT_8BIT.addLongTrieCodedDocumentField(doc, "exampleLong", lvalue, true /* index the field */, Field.Store.YES);
-  Date dvalue = new Date(); // actual time
-  TrieUtils.VARIANT_8BIT.addDateTrieCodedDocumentField(doc, "exampleDate", dvalue, true /* index the field */, Field.Store.YES);
-  // add document to IndexWriter
+  TrieUtils.addIndexedFields(doc, "exampleLong", TrieUtils.trieCodeLong(lvalue, precisionStep));
+  double dvalue = 1.057E17;
+  TrieUtils.addIndexedFields(doc, "exampleDouble", TrieUtils.trieCodeLong(TrieUtils.doubleToSortableLong(dvalue), precisionStep));
+  int ivalue = 121345;
+  TrieUtils.addIndexedFields(doc, "exampleInt", TrieUtils.trieCodeInt(ivalue, precisionStep));
+  float fvalue = 1.057E17f;
+  TrieUtils.addIndexedFields(doc, "exampleFloat", TrieUtils.trieCodeInt(TrieUtils.floatToSortableInt(fvalue), precisionStep));
+  Date datevalue = new Date(); <em>// actual time</em>
+  TrieUtils.addIndexedFields(doc, "exampleDate", TrieUtils.trieCodeLong(datevalue.getTime(), precisionStep));
+  
+  <em>// if you want to also store one of the values:</em>
+  doc.add(new Field("exampleLong", Long.toString(lvalue), Field.Store.YES, Field.Index.NO));
+  
+  <em>// or as encoded value:</em>
+  doc.add(new Field("exampleLong2", TrieUtils.longToPrefixCoded(lvalue), Field.Store.YES, Field.Index.NO));
+  
+  <em>// now add document to IndexWriter, as usual</em>
 </pre>
 
-<p>The numeric index fields you prepared in this way can be searched by {@link org.apache.lucene.search.trie.TrieRangeQuery}:</p>
+<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>
 
 <pre>
-  // Java 1.4, because Double.valueOf(double) is not available:
-  Query q = new TrieRangeQuery("exampleDouble", new Double(1.0E17), new Double(2.0E17), TrieUtils.VARIANT_8BIT);
-  // OR, Java 1.5, using autoboxing:
-  Query q = new TrieRangeQuery("exampleDouble", 1.0E17, 2.0E17, TrieUtils.VARIANT_8BIT);
+  <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();
+  
+  <em>// OR, Java 1.5, using autoboxing:</em>
+  Query q = new LongTrieRangeFilter("exampleLong", precisionStep, 123L, 999999L, true, true).asQuery();
+  
+  <em>// execute the search, as usual:</em>
   TopDocs docs = searcher.search(q, 10);
   for (int i = 0; i&lt;docs.scoreDocs.length; i++) {
     Document doc = searcher.doc(docs.scoreDocs[i].doc);
     System.out.println(doc.get("exampleString"));
-    // decode the stored numerical value (important!!!):
-    System.out.println(TrieUtils.VARIANT_8BIT.trieCodedToDouble(doc.get("exampleDouble")));
+    
+    <em>// decode a prefix coded, stored field:</em>
+    System.out.println(TrieUtils.prefixCodedToLong(doc.get("exampleLong2")));
   }
 </pre>
 
@@ -82,9 +112,9 @@
 <p>Comparisions of the different types of RangeQueries on an index with about 500,000 docs showed
 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
-{@link org.apache.lucene.search.trie.TrieRangeQuery} took &lt;100ms to
-complete (on an Opteron64 machine, Java 1.5, {@link org.apache.lucene.search.trie.TrieUtils#VARIANT_8BIT}).
+{@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
+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>
 

Added: lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestIntTrieRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestIntTrieRangeFilter.java?rev=744207&view=auto
==============================================================================
--- lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestIntTrieRangeFilter.java (added)
+++ lucene/java/trunk/contrib/queries/src/test/org/apache/lucene/search/trie/TestIntTrieRangeFilter.java Fri Feb 13 18:27:01 2009
@@ -0,0 +1,291 @@
+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.util.Random;
+
+import org.apache.lucene.analysis.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriter.MaxFieldLength;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.RangeQuery;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestIntTrieRangeFilter extends LuceneTestCase
+{
+  // distance of entries
+  private static final int distance = 6666;
+  // shift the starting of the values to the left, to also have negative values:
+  private static final int startOffset = - 1 << 15;
+  // number of docs to generate for testing
+  private static final int noDocs = 10000;
+  
+  private static final RAMDirectory directory;
+  private static final IndexSearcher searcher;
+  static {
+    try {    
+      directory = new RAMDirectory();
+      IndexWriter writer = new IndexWriter(directory, new WhitespaceAnalyzer(),
+      true, MaxFieldLength.UNLIMITED);
+      
+      // Add a series of noDocs docs with increasing int values
+      for (int l=0; l<noDocs; l++) {
+        Document doc=new Document();
+        // add fields, that have a distance to test general functionality
+        final int val=distance*l+startOffset;
+        TrieUtils.addIndexedFields(doc,"field8", TrieUtils.trieCodeInt(val, 8));
+        doc.add(new Field("field8", TrieUtils.intToPrefixCoded(val), Field.Store.YES, Field.Index.NO));
+        TrieUtils.addIndexedFields(doc,"field4", TrieUtils.trieCodeInt(val, 4));
+        doc.add(new Field("field4", TrieUtils.intToPrefixCoded(val), Field.Store.YES, Field.Index.NO));
+        TrieUtils.addIndexedFields(doc,"field2", TrieUtils.trieCodeInt(val, 2));
+        doc.add(new Field("field2", TrieUtils.intToPrefixCoded(val), Field.Store.YES, Field.Index.NO));
+        // add ascending fields with a distance of 1, beginning at -noDocs/2 to test the correct splitting of range and inclusive/exclusive
+        TrieUtils.addIndexedFields(doc,"ascfield8", TrieUtils.trieCodeInt(l-(noDocs/2), 8));
+        TrieUtils.addIndexedFields(doc,"ascfield4", TrieUtils.trieCodeInt(l-(noDocs/2), 4));
+        TrieUtils.addIndexedFields(doc,"ascfield2", TrieUtils.trieCodeInt(l-(noDocs/2), 2));
+        writer.addDocument(doc);
+      }
+    
+      writer.optimize();
+      writer.close();
+      searcher=new IndexSearcher(directory);
+    } catch (Exception e) {
+      throw new Error(e);
+    }
+  }
+  
+  private void testRange(int precisionStep) throws Exception {
+    String field="field"+precisionStep;
+    int count=3000;
+    int lower=(distance*3/2)+startOffset, upper=lower + count*distance + (distance/3);
+    IntTrieRangeFilter f=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), true, true);
+    TopDocs topDocs = searcher.search(f.asQuery(), null, noDocs, Sort.INDEXORDER);
+    System.out.println("Found "+f.getLastNumberOfTerms()+" distinct terms in range for field '"+field+"'.");
+    ScoreDoc[] sd = topDocs.scoreDocs;
+    assertNotNull(sd);
+    assertEquals("Score doc count", count, sd.length );
+    Document doc=searcher.doc(sd[0].doc);
+    assertEquals("First doc", 2*distance+startOffset, TrieUtils.prefixCodedToInt(doc.get(field)) );
+    doc=searcher.doc(sd[sd.length-1].doc);
+    assertEquals("Last doc", (1+count)*distance+startOffset, TrieUtils.prefixCodedToInt(doc.get(field)) );
+  }
+
+  public void testRange_8bit() throws Exception {
+    testRange(8);
+  }
+  
+  public void testRange_4bit() throws Exception {
+    testRange(4);
+  }
+  
+  public void testRange_2bit() throws Exception {
+    testRange(2);
+  }
+  
+  private void testLeftOpenRange(int precisionStep) throws Exception {
+    String field="field"+precisionStep;
+    int count=3000;
+    int upper=(count-1)*distance + (distance/3) + startOffset;
+    IntTrieRangeFilter f=new IntTrieRangeFilter(field, precisionStep, null, new Integer(upper), true, true);
+    TopDocs topDocs = searcher.search(f.asQuery(), null, noDocs, Sort.INDEXORDER);
+    System.out.println("Found "+f.getLastNumberOfTerms()+" distinct terms in left open range for field '"+field+"'.");
+    ScoreDoc[] sd = topDocs.scoreDocs;
+    assertNotNull(sd);
+    assertEquals("Score doc count", count, sd.length );
+    Document doc=searcher.doc(sd[0].doc);
+    assertEquals("First doc", startOffset, TrieUtils.prefixCodedToInt(doc.get(field)) );
+    doc=searcher.doc(sd[sd.length-1].doc);
+    assertEquals("Last doc", (count-1)*distance+startOffset, TrieUtils.prefixCodedToInt(doc.get(field)) );
+  }
+  
+  public void testLeftOpenRange_8bit() throws Exception {
+    testLeftOpenRange(8);
+  }
+  
+  public void testLeftOpenRange_4bit() throws Exception {
+    testLeftOpenRange(4);
+  }
+  
+  public void testLeftOpenRange_2bit() throws Exception {
+    testLeftOpenRange(2);
+  }
+  
+  private void testRightOpenRange(int precisionStep) throws Exception {
+    String field="field"+precisionStep;
+    int count=3000;
+    int lower=(count-1)*distance + (distance/3) +startOffset;
+    IntTrieRangeFilter f=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), null, true, true);
+    TopDocs topDocs = searcher.search(f.asQuery(), null, noDocs, Sort.INDEXORDER);
+    System.out.println("Found "+f.getLastNumberOfTerms()+" distinct terms in right open range for field '"+field+"'.");
+    ScoreDoc[] sd = topDocs.scoreDocs;
+    assertNotNull(sd);
+    assertEquals("Score doc count", noDocs-count, sd.length );
+    Document doc=searcher.doc(sd[0].doc);
+    assertEquals("First doc", count*distance+startOffset, TrieUtils.prefixCodedToInt(doc.get(field)) );
+    doc=searcher.doc(sd[sd.length-1].doc);
+    assertEquals("Last doc", (noDocs-1)*distance+startOffset, TrieUtils.prefixCodedToInt(doc.get(field)) );
+  }
+  
+  public void testRightOpenRange_8bit() throws Exception {
+    testRightOpenRange(8);
+  }
+  
+  public void testRightOpenRange_4bit() throws Exception {
+    testRightOpenRange(4);
+  }
+  
+  public void testRightOpenRange_2bit() throws Exception {
+    testRightOpenRange(2);
+  }
+  
+  private void testRandomTrieAndClassicRangeQuery(int precisionStep) throws Exception {
+    final Random rnd=newRandom();
+    String field="field"+precisionStep;
+    // 50 random tests, the tests may also return 0 results, if min>max, but this is ok
+    for (int i=0; i<50; i++) {
+      int lower=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
+      int upper=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
+      // test inclusive range
+      Query tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), true, true).asQuery();
+      RangeQuery cq=new RangeQuery(field, TrieUtils.intToPrefixCoded(lower), TrieUtils.intToPrefixCoded(upper), true, true);
+      cq.setConstantScoreRewrite(true);
+      TopDocs tTopDocs = searcher.search(tq, 1);
+      TopDocs cTopDocs = searcher.search(cq, 1);
+      assertEquals("Returned count for IntTrieRangeFilter and RangeQuery must be equal", cTopDocs.totalHits, tTopDocs.totalHits );
+      // test exclusive range
+      tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), false, false).asQuery();
+      cq=new RangeQuery(field, TrieUtils.intToPrefixCoded(lower), TrieUtils.intToPrefixCoded(upper), false, false);
+      cq.setConstantScoreRewrite(true);
+      tTopDocs = searcher.search(tq, 1);
+      cTopDocs = searcher.search(cq, 1);
+      assertEquals("Returned count for IntTrieRangeFilter and RangeQuery must be equal", cTopDocs.totalHits, tTopDocs.totalHits );
+      // test left exclusive range
+      tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), false, true).asQuery();
+      cq=new RangeQuery(field, TrieUtils.intToPrefixCoded(lower), TrieUtils.intToPrefixCoded(upper), false, true);
+      cq.setConstantScoreRewrite(true);
+      tTopDocs = searcher.search(tq, 1);
+      cTopDocs = searcher.search(cq, 1);
+      assertEquals("Returned count for IntTrieRangeFilter and RangeQuery must be equal", cTopDocs.totalHits, tTopDocs.totalHits );
+      // test right exclusive range
+      tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), true, false).asQuery();
+      cq=new RangeQuery(field, TrieUtils.intToPrefixCoded(lower), TrieUtils.intToPrefixCoded(upper), true, false);
+      cq.setConstantScoreRewrite(true);
+      tTopDocs = searcher.search(tq, 1);
+      cTopDocs = searcher.search(cq, 1);
+      assertEquals("Returned count for IntTrieRangeFilter and RangeQuery must be equal", cTopDocs.totalHits, tTopDocs.totalHits );
+    }
+  }
+  
+  public void testRandomTrieAndClassicRangeQuery_8bit() throws Exception {
+    testRandomTrieAndClassicRangeQuery(8);
+  }
+  
+  public void testRandomTrieAndClassicRangeQuery_4bit() throws Exception {
+    testRandomTrieAndClassicRangeQuery(4);
+  }
+  
+  public void testRandomTrieAndClassicRangeQuery_2bit() throws Exception {
+    testRandomTrieAndClassicRangeQuery(2);
+  }
+  
+  private void testRangeSplit(int precisionStep) throws Exception {
+    final Random rnd=newRandom();
+    String field="ascfield"+precisionStep;
+    // 50 random tests
+    for (int i=0; i<50; i++) {
+      int lower=(int)(rnd.nextDouble()*noDocs - noDocs/2);
+      int upper=(int)(rnd.nextDouble()*noDocs - noDocs/2);
+      if (lower>upper) {
+        int a=lower; lower=upper; upper=a;
+      }
+      // test inclusive range
+      Query tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), true, true).asQuery();
+      TopDocs tTopDocs = searcher.search(tq, 1);
+      assertEquals("Returned count of range query must be equal to inclusive range length", upper-lower+1, tTopDocs.totalHits );
+      // test exclusive range
+      tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), false, false).asQuery();
+      tTopDocs = searcher.search(tq, 1);
+      assertEquals("Returned count of range query must be equal to exclusive range length", Math.max(upper-lower-1, 0), tTopDocs.totalHits );
+      // test left exclusive range
+      tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), false, true).asQuery();
+      tTopDocs = searcher.search(tq, 1);
+      assertEquals("Returned count of range query must be equal to half exclusive range length", upper-lower, tTopDocs.totalHits );
+      // test right exclusive range
+      tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), true, false).asQuery();
+      tTopDocs = searcher.search(tq, 1);
+      assertEquals("Returned count of range query must be equal to half exclusive range length", upper-lower, tTopDocs.totalHits );
+    }
+  }
+
+  public void testRangeSplit_8bit() throws Exception {
+    testRangeSplit(8);
+  }
+  
+  public void testRangeSplit_4bit() throws Exception {
+    testRangeSplit(4);
+  }
+  
+  public void testRangeSplit_2bit() throws Exception {
+    testRangeSplit(2);
+  }
+  
+  private void testSorting(int precisionStep) throws Exception {
+    final Random rnd=newRandom();
+    String field="field"+precisionStep;
+    // 10 random tests, the index order is ascending,
+    // so using a reverse sort field should retun descending documents
+    for (int i=0; i<10; i++) {
+      int lower=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
+      int upper=(int)(rnd.nextDouble()*noDocs*distance)+startOffset;
+      if (lower>upper) {
+        int a=lower; lower=upper; upper=a;
+      }
+      Query tq=new IntTrieRangeFilter(field, precisionStep, new Integer(lower), new Integer(upper), true, true).asQuery();
+      TopDocs topDocs = searcher.search(tq, null, noDocs, new Sort(TrieUtils.getIntSortField(field, true)));
+      if (topDocs.totalHits==0) continue;
+      ScoreDoc[] sd = topDocs.scoreDocs;
+      assertNotNull(sd);
+      int last=TrieUtils.prefixCodedToInt(searcher.doc(sd[0].doc).get(field));
+      for (int j=1; j<sd.length; j++) {
+        int act=TrieUtils.prefixCodedToInt(searcher.doc(sd[j].doc).get(field));
+        assertTrue("Docs should be sorted backwards", last>act );
+        last=act;
+      }
+    }
+  }
+
+  public void testSorting_8bit() throws Exception {
+    testSorting(8);
+  }
+  
+  public void testSorting_4bit() throws Exception {
+    testSorting(4);
+  }
+  
+  public void testSorting_2bit() throws Exception {
+    testSorting(2);
+  }
+  
+}