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 mi...@apache.org on 2009/11/08 12:53:13 UTC

svn commit: r833860 - in /lucene/java/branches/flex_1458: contrib/benchmark/ contrib/memory/src/java/org/apache/lucene/index/memory/ contrib/queries/src/java/org/apache/lucene/search/ contrib/regex/src/java/org/apache/lucene/search/regex/ src/java/org/...

Author: mikemccand
Date: Sun Nov  8 11:53:13 2009
New Revision: 833860

URL: http://svn.apache.org/viewvc?rev=833860&view=rev
Log:
LUCENE-1458 (on flex branch): fix silly problem with MTQ (we were not stopping at the end term for FilteredTermsEnums that ended)

Modified:
    lucene/java/branches/flex_1458/contrib/benchmark/sortBench.py
    lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
    lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/RegexTermsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/NumericRangeQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixTermsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/WildcardTermsEnum.java

Modified: lucene/java/branches/flex_1458/contrib/benchmark/sortBench.py
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/benchmark/sortBench.py?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/benchmark/sortBench.py (original)
+++ lucene/java/branches/flex_1458/contrib/benchmark/sortBench.py Sun Nov  8 11:53:13 2009
@@ -495,12 +495,12 @@
 
   queries = (
     'body:[tec TO tet]',
+    'real*',
     '1',
     '2',
     '+1 +2',
     '+1 -2',
     '1 2 3 -4',
-    'real*',
     '"world economy"')
 
   for query in queries:
@@ -526,6 +526,7 @@
       baseline = r.runOne(TRUNK_DIR, s, 'baseline_%s' % prefix, maxDocs, numDocs, query, verify=doVerify)
 
       # flex
+      indexPath = '%s/%s' % (INDEX_DIR_BASE, indexes['flex'])
       s = r.getAlg(indexPath,
                    'Search',
                    numHits,
@@ -538,7 +539,7 @@
 
       verify(baseline, flex)
 
-      if mode == 'run':
+      if mode == 'run' and not DEBUG:
         r.compare(baseline, flex,
                   query, deletePct, baseline[0])
         r.save(name)

Modified: lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/java/branches/flex_1458/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Sun Nov  8 11:53:13 2009
@@ -828,8 +828,13 @@
 
       public MemoryTermsEnum(Info info) {
         this.info = info;
-        info.sortTerms();
-        this.sortedTerms = info.sortedTerms;
+        // nocommit -- understand why this null check became necessary?
+        if (info != null) {
+          info.sortTerms();
+          this.sortedTerms = info.sortedTerms;
+        } else {
+          sortedTerms = new Map.Entry[0];
+        }
       }
 
       public TermRef next() {

Modified: lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/queries/src/java/org/apache/lucene/search/FuzzyLikeThisQuery.java Sun Nov  8 11:53:13 2009
@@ -200,7 +200,8 @@
                   int df = reader.docFreq(startTerm);
                   int numVariants=0;
                   int totalVariantDocFreqs=0;
-                  do {
+                  if (!fe.empty()) {
+                    do {
                       TermRef possibleMatch = fe.term();
                       if (possibleMatch!=null) {
                         numVariants++;
@@ -212,8 +213,8 @@
                           minScore = ((ScoreTerm)variantsQ.top()).score; // maintain minScore
                         }
                       }
-                    }
-                  while(fe.next() != null);
+                    } while(fe.next() != null);
+                  }
 
                   if(numVariants>0)
                     {

Modified: lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/RegexTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/RegexTermsEnum.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/RegexTermsEnum.java (original)
+++ lucene/java/branches/flex_1458/contrib/regex/src/java/org/apache/lucene/search/regex/RegexTermsEnum.java Sun Nov  8 11:53:13 2009
@@ -67,11 +67,11 @@
     return field;
   }
 
-  protected final boolean accept(TermRef term) {
+  protected final AcceptStatus accept(TermRef term) {
     if (term.startsWith(prefixRef)) {
-      return regexImpl.match(term.toString());
+      return regexImpl.match(term.toString()) ? AcceptStatus.YES : AcceptStatus.NO;
     } else {
-      return false;
+      return AcceptStatus.END;
     }
   }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java Sun Nov  8 11:53:13 2009
@@ -1384,16 +1384,17 @@
         final SeekStatus status = subs[i].terms.seek(term);
         if (status == SeekStatus.FOUND) {
           top[numTop++] = subs[i];
-          current = subs[i].current = term;
+          subs[i].current = term;
         } else if (status == SeekStatus.NOT_FOUND) {
           queue.add(subs[i]);
-          current = subs[i].current = subs[i].terms.term();
+          subs[i].current = subs[i].terms.term();
         } else {
           // enum exhausted
         }
       }
 
       if (numTop > 0) {
+        current = term;
         return SeekStatus.FOUND;
       } else if (queue.size() > 0) {
         pullTop();

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java Sun Nov  8 11:53:13 2009
@@ -1095,7 +1095,7 @@
    * (transactional semantics).
    * @throws IOException if there is a low-level IO error
    */
-  protected final synchronized void commit(Map<String, String> commitUserData) throws IOException {
+  public final synchronized void commit(Map<String, String> commitUserData) throws IOException {
     if (hasChanges) {
       doCommit(commitUserData);
     }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java Sun Nov  8 11:53:13 2009
@@ -263,6 +263,10 @@
           System.out.println("[" + desc + "] dr.init freqIn seek " + freqOffset + " this=" + this + " (in=" + freqIn + "; this=" + this + ") docFreq=" + TermsDictReader.this.docFreq);
         }
         this.skipDocs = skipDocs;
+        // nocommit this seek frequently isn't needed, when
+        // we enum terms and all docs for each term (MTQ,
+        // or, merging).  is this seek costing us anything?
+        // we should avoid it so...
         freqIn.seek(freqOffset);
         this.docFreq = TermsDictReader.this.docFreq;
         count = 0;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java Sun Nov  8 11:53:13 2009
@@ -220,7 +220,7 @@
       threadResources.close();
     }
     
-    private ThreadResources getThreadResources() throws IOException {
+    protected ThreadResources getThreadResources() throws IOException {
       ThreadResources resources = (ThreadResources) threadResources.get();
       if (resources == null) {
         // Cache does not have to be thread-safe, it is only used by one thread at the same time

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java Sun Nov  8 11:53:13 2009
@@ -34,12 +34,14 @@
  * greater than all that precede it.</p>
 */
 public abstract class FilteredTermsEnum extends TermsEnum {
+
+  protected static enum AcceptStatus {YES, NO, END};
     
   /** the delegate enum - to set this member use {@link #setEnum} */
   protected TermsEnum actualEnum;
     
   /** Return true if term is acceptd */
-  protected abstract boolean accept(TermRef term);
+  protected abstract AcceptStatus accept(TermRef term);
     
   /** Equality measure on the term */
   public abstract float difference();
@@ -65,8 +67,11 @@
       if (status == SeekStatus.END) {
         return null;
       } else {
-        if (!accept(actualEnum.term())) {
+        AcceptStatus s = accept(actualEnum.term());
+        if (s == AcceptStatus.NO) {
           return next();
+        } else if (s == AcceptStatus.END) {
+          return null;
         } else {
           return actualEnum.term();
         }
@@ -92,15 +97,20 @@
     return actualEnum.docFreq();
   }
     
-  /** Increments the enumeration to the next element.  True if one exists. */
+  /** Increments the enumeration to the next element.
+   * Non-null if one exists, or null if it's the end. */
   @Override
   public TermRef next() throws IOException {
     assert actualEnum != null;
     while (true) {
       TermRef term = actualEnum.next();
       if (term != null) {
-        if (accept(term)) {
+        AcceptStatus s = accept(term);
+        if (s == AcceptStatus.YES) {
           return term;
+        } else if (s == AcceptStatus.END) {
+          // end
+          return null;
         }
       } else {
         // end
@@ -122,13 +132,16 @@
   private SeekStatus finishSeek(SeekStatus status) throws IOException {
     if (status != SeekStatus.END) {
       TermRef term = actualEnum.term();
-      if (!accept(term)) {
+      final AcceptStatus s = accept(term);
+      if (s == AcceptStatus.NO) {
         term = next();
         if (term == null) {
           return SeekStatus.END;
         } else {
           return SeekStatus.NOT_FOUND;
         }
+      } else if (s == AcceptStatus.END) {
+        return SeekStatus.END;
       } else {
         return status;
       }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Sun Nov  8 11:53:13 2009
@@ -147,14 +147,14 @@
    * calculate the distance between the given term and the comparing term. 
    */
   @Override
-  protected final boolean accept(TermRef term) {
+  protected final AcceptStatus accept(TermRef term) {
     if (term.startsWith(prefixTermRef)) {
       // TODO: costly that we create intermediate String:
       final String target = term.toString().substring(prefix.length());
       this.similarity = similarity(target);
-      return (similarity > minimumSimilarity);
+      return (similarity > minimumSimilarity) ? AcceptStatus.YES : AcceptStatus.NO;
     } else {
-      return false;
+      return AcceptStatus.END;
     }
   }
   

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=833860&r1=833859&r2=833860&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 Sun Nov  8 11:53:13 2009
@@ -671,11 +671,15 @@
      * and forwards to the next sub-range.
      */
     @Override
-    protected boolean accept(TermRef term) {
-      return (term.compareTerm(currentUpperBound) <= 0);
+    protected AcceptStatus accept(TermRef term) {
+      if (term.compareTerm(currentUpperBound) <= 0) {
+        return AcceptStatus.YES;
+      } else {
+        return AcceptStatus.NO;
+      }
     }
 
-    /** Increments the enumeration to the next element.  True if one exists. */
+    /** Increments the enumeration to the next element.  Non-null if one exists. */
     @Override
     public TermRef next() throws IOException {
       //System.out.println("nrq.next");
@@ -683,7 +687,7 @@
       // next term, if no such term exists, fall-through
       if (actualEnum != null) {
         TermRef term = actualEnum.next();
-        if (term != null && accept(term)) {
+        if (term != null && accept(term) == AcceptStatus.YES) {
           //System.out.println("  return term=" + term.toBytesString());
           return term;
         }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixTermsEnum.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixTermsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixTermsEnum.java Sun Nov  8 11:53:13 2009
@@ -70,7 +70,11 @@
   }
 
   @Override
-  protected boolean accept(TermRef term) {
-    return term.startsWith(prefixRef);
+  protected AcceptStatus accept(TermRef term) {
+    if (term.startsWith(prefixRef)) {
+      return AcceptStatus.YES;
+    } else {
+      return AcceptStatus.END;
+    }
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeTermsEnum.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeTermsEnum.java Sun Nov  8 11:53:13 2009
@@ -36,7 +36,6 @@
 public class TermRangeTermsEnum extends FilteredTermsEnum {
 
   private Collator collator;
-  private boolean end;
   private String field;
   private String upperTermText;
   private String lowerTermText;
@@ -88,7 +87,7 @@
       this.includeLower = true;
     }
     lowerTermRef = new TermRef(this.lowerTermText);
-    
+
     if (this.upperTermText == null) {
       this.includeUpper = true;
       upperTermRef = null;
@@ -127,7 +126,7 @@
   }
 
   @Override
-  protected boolean accept(TermRef term) {
+  protected AcceptStatus accept(TermRef term) {
     if (collator == null) {
       // Use Unicode code point ordering
       if (upperTermRef != null) {
@@ -138,10 +137,10 @@
          */
         if ((cmp < 0) ||
             (!includeUpper && cmp==0)) {
-          return false;
+          return AcceptStatus.END;
         }
       }
-      return true;
+      return AcceptStatus.YES;
     } else {
       if ((includeLower
            ? collator.compare(term.toString(), lowerTermText) >= 0
@@ -150,10 +149,9 @@
               || (includeUpper
                   ? collator.compare(term.toString(), upperTermText) <= 0
                   : collator.compare(term.toString(), upperTermText) < 0))) {
-        return true;
+        return AcceptStatus.YES;
       }
-      end = true;
+      return AcceptStatus.NO;
     }
-    return false;
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/WildcardTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/WildcardTermsEnum.java?rev=833860&r1=833859&r2=833860&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/WildcardTermsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/WildcardTermsEnum.java Sun Nov  8 11:53:13 2009
@@ -83,15 +83,20 @@
   }
 
   @Override
-  protected final boolean accept(TermRef term) {
+  protected final AcceptStatus accept(TermRef term) {
     if (term.startsWith(preTermRef)) {
       // TODO: would be better, but trickier, to not have to
       // build intermediate String (ie check wildcard matching
       // directly on UTF8)
       final String searchText = term.toString();
-      return wildcardEquals(text, 0, searchText, preLen);
+      if (wildcardEquals(text, 0, searchText, preLen)) {
+        return AcceptStatus.YES;
+      } else {
+        return AcceptStatus.NO;
+      }
+    } else {
+      return AcceptStatus.END;
     }
-    return false;
   }
 
   @Override