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/11/30 10:13:12 UTC

svn commit: r885360 - /lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/NumericRangeQuery.java

Author: uschindler
Date: Mon Nov 30 09:13:11 2009
New Revision: 885360

URL: http://svn.apache.org/viewvc?rev=885360&view=rev
Log:
Simplify and optimize NumericRangeTermEnum:
- the range split logic only seeks forward (an assert verifies this), so the iterator can be reused (like Automaton)
- removed the iteration by not using setEnum() [throws UOE], see LUCENE-2087
- removed TermEnum, as class cannot be subclassed; getEnum() throws UOE.

Modified:
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/NumericRangeQuery.java

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=885360&r1=885359&r2=885360&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/NumericRangeQuery.java Mon Nov 30 09:13:11 2009
@@ -29,6 +29,7 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermRef;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 
 /**
  * <p>A {@link Query} that matches numeric values within a
@@ -304,7 +305,7 @@
   
   @Override
   protected FilteredTermEnum getEnum(final IndexReader reader) throws IOException {
-    return new NumericRangeTermEnum(reader);
+    throw new UnsupportedOperationException("not implemented");
   }
 
   @Override
@@ -383,175 +384,6 @@
   final boolean minInclusive,maxInclusive;
 
   /**
-   * Subclass of FilteredTermEnum for enumerating all terms that match the
-   * sub-ranges for trie range queries.
-   * <p>
-   * WARNING: This term enumeration is not guaranteed to be always ordered by
-   * {@link Term#compareTo}.
-   * The ordering depends on how {@link NumericUtils#splitLongRange} and
-   * {@link NumericUtils#splitIntRange} generates the sub-ranges. For
-   * {@link MultiTermQuery} ordering is not relevant.
-   *
-   * @deprecated use NumericRangeTermsEnum instead
-   */
-  // nocommit -- can we remove this?  only back compat
-  // concern would be subclasses of NRQ that invoke getEnum
-  private final class NumericRangeTermEnum extends FilteredTermEnum {
-
-    private final IndexReader reader;
-    private final LinkedList<String> rangeBounds = new LinkedList<String>();
-    private String currentUpperBound = null;
-
-    NumericRangeTermEnum(final IndexReader reader) throws IOException {
-      this.reader = reader;
-      
-      switch (valSize) {
-        case 64: {
-          // lower
-          long minBound = Long.MIN_VALUE;
-          if (min instanceof Long) {
-            minBound = min.longValue();
-          } else if (min instanceof Double) {
-            minBound = NumericUtils.doubleToSortableLong(min.doubleValue());
-          }
-          if (!minInclusive && min != null) {
-            if (minBound == Long.MAX_VALUE) break;
-            minBound++;
-          }
-          
-          // upper
-          long maxBound = Long.MAX_VALUE;
-          if (max instanceof Long) {
-            maxBound = max.longValue();
-          } else if (max instanceof Double) {
-            maxBound = NumericUtils.doubleToSortableLong(max.doubleValue());
-          }
-          if (!maxInclusive && max != null) {
-            if (maxBound == Long.MIN_VALUE) break;
-            maxBound--;
-          }
-          
-          NumericUtils.splitLongRange(new NumericUtils.LongRangeBuilder() {
-            @Override
-            public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
-              rangeBounds.add(minPrefixCoded);
-              rangeBounds.add(maxPrefixCoded);
-            }
-          }, precisionStep, minBound, maxBound);
-          break;
-        }
-          
-        case 32: {
-          // lower
-          int minBound = Integer.MIN_VALUE;
-          if (min instanceof Integer) {
-            minBound = min.intValue();
-          } else if (min instanceof Float) {
-            minBound = NumericUtils.floatToSortableInt(min.floatValue());
-          }
-          if (!minInclusive && min != null) {
-            if (minBound == Integer.MAX_VALUE) break;
-            minBound++;
-          }
-          
-          // upper
-          int maxBound = Integer.MAX_VALUE;
-          if (max instanceof Integer) {
-            maxBound = max.intValue();
-          } else if (max instanceof Float) {
-            maxBound = NumericUtils.floatToSortableInt(max.floatValue());
-          }
-          if (!maxInclusive && max != null) {
-            if (maxBound == Integer.MIN_VALUE) break;
-            maxBound--;
-          }
-          
-          NumericUtils.splitIntRange(new NumericUtils.IntRangeBuilder() {
-            @Override
-            public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
-              rangeBounds.add(minPrefixCoded);
-              rangeBounds.add(maxPrefixCoded);
-            }
-          }, precisionStep, minBound, maxBound);
-          break;
-        }
-          
-        default:
-          // should never happen
-          throw new IllegalArgumentException("valSize must be 32 or 64");
-      }
-      
-      // seek to first term
-      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() == 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 = rangeBounds.removeFirst();
-      this.currentUpperBound = 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(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();
-    }
-
-  }
-
-
-  /**
    * Subclass of FilteredTermsEnum for enumerating all terms that match the
    * sub-ranges for trie range queries, using flex API.
    * <p>
@@ -599,7 +431,6 @@
           }
           
           NumericUtils.splitLongRange(new NumericUtils.LongRangeBuilder() {
-            //@Override
             @Override
             public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
               rangeBounds.add(minPrefixCoded);
@@ -635,7 +466,6 @@
           }
           
           NumericUtils.splitIntRange(new NumericUtils.IntRangeBuilder() {
-            //@Override
             @Override
             public final void addRange(String minPrefixCoded, String maxPrefixCoded) {
               rangeBounds.add(minPrefixCoded);
@@ -649,17 +479,18 @@
           // should never happen
           throw new IllegalArgumentException("valSize must be 32 or 64");
       }
-      
-      // TODO: NRQ by design relies on a specific sort
-      // order; I think UT8 or UTF16 would work (NRQ encodes
-      // to only ASCII).
-      
-      Terms terms = reader.fields().terms(field);
+
+      // initialize iterator
+      final Terms terms = reader.fields().terms(field);
       if (terms != null) {
-        // cache locally
+        // TODO: NRQ by design relies on a specific sort
+        // order; I think UT8 or UTF16 would work (NRQ encodes
+        // to only ASCII).
         termComp = terms.getTermComparator();
+        actualEnum = terms.iterator();
       } else {
         termComp = null;
+        actualEnum = null;
       }
 
       // seek to first term
@@ -677,66 +508,61 @@
       return empty;
     }
 
+    /** this is a dummy, it is not used by this class. */
+    @Override
+    protected TermRef setEnum(TermsEnum actualEnum, TermRef term) throws IOException {
+      throw new UnsupportedOperationException("not implemented");
+    }
+    
     @Override
     public String field() {
       return field;
     }
 
-    /**
-     * 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 AcceptStatus accept(TermRef term) {
-      if (termComp.compare(term, currentUpperBound) <= 0) {
-        return AcceptStatus.YES;
-      } else {
-        return AcceptStatus.NO;
-      }
+      return (termComp.compare(term, currentUpperBound) <= 0) ?
+        AcceptStatus.YES : AcceptStatus.NO;
     }
 
-    /** Increments the enumeration to the next element.  Non-null if one exists. */
     @Override
     public TermRef next() throws IOException {
-      //System.out.println("nrq.next");
-      // if the actual enum is initialized, try change to
-      // next term, if no such term exists, fall-through
-      if (actualEnum != null) {
+      if (actualEnum == null)
+        return null;
+      
+      // try change to next term, if no such term exists, fall-through
+      // (we can only do this if the enum was already seeked)
+      if (currentUpperBound != null) {
         TermRef term = actualEnum.next();
         if (term != null && accept(term) == AcceptStatus.YES) {
-          //System.out.println("  return term=" + term.toBytesString());
           return term;
         }
       }
 
-      //System.out.println("  ranges = " + rangeBounds.size());
-
-      // if all above fails, we go forward to the next enum,
-      // if one is available
-      if (rangeBounds.size() < 2) {
-        assert rangeBounds.size() == 0;
-        //System.out.println("  return null0");
-        return null;
-      }
-
-      final TermRef lowerBound = new TermRef(rangeBounds.removeFirst());
-      this.currentUpperBound = new TermRef(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!
-      Terms terms = reader.fields().terms(field);
-      if (terms != null) {
-        return setEnum(terms.iterator(), lowerBound);
-      } else {
-        //System.out.println("  return null");
-        return null;
+      // if all above fails, we seek forward
+      while (rangeBounds.size() >= 2) {
+        assert rangeBounds.size() % 2 == 0;
+
+        final TermRef lowerBound = new TermRef(rangeBounds.removeFirst());
+        assert currentUpperBound == null || termComp.compare(currentUpperBound, lowerBound) <= 0 :
+          "The current upper bound must be <= the new lower bound";
+        
+        this.currentUpperBound = new TermRef(rangeBounds.removeFirst());
+
+        SeekStatus status = actualEnum.seek(lowerBound);
+        if (status != SeekStatus.END) {
+          final TermRef term = actualEnum.term();
+          if (accept(term) == AcceptStatus.YES) {
+            return term;
+          }
+        }
       }
+      
+      // no more sub-range enums available
+      assert rangeBounds.size() == 0;
+      return null;
     }
+
   }
   
 }