You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2012/10/26 17:25:42 UTC

svn commit: r1402547 - in /lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search: ./ positions/

Author: simonw
Date: Fri Oct 26 15:25:42 2012
New Revision: 1402547

URL: http://svn.apache.org/viewvc?rev=1402547&view=rev
Log:
LUCENE-2878: javadocs and cleanups

Modified:
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/BooleanIntervalIterator.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionIntervalIterator.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionIntervalIterator.java
    lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/TermIntervalIterator.java

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri Oct 26 15:25:42 2012
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
 
-import org.apache.lucene.search.positions.BooleanIntervalIterator;
 import org.apache.lucene.search.positions.ConjunctionIntervalIterator;
 import org.apache.lucene.search.positions.IntervalIterator;
 import org.apache.lucene.util.ArrayUtil;
@@ -150,7 +149,7 @@ class ConjunctionScorer extends Scorer {
       throw new IllegalStateException("no positions requested for this scorer");
     }
       // only created if needed for this scorer - no penalty for non-positional queries
-    return new ConjunctionIntervalIterator(this, collectPositions, BooleanIntervalIterator.pullIterators(collectPositions, scorersOrdered));
+    return new ConjunctionIntervalIterator(this, collectPositions, pullIterators(collectPositions, scorersOrdered));
   }
 
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Fri Oct 26 15:25:42 2012
@@ -16,7 +16,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.search.positions.BooleanIntervalIterator;
 import org.apache.lucene.search.positions.DisjunctionIntervalIterator;
 import org.apache.lucene.search.positions.IntervalIterator;
 
@@ -140,6 +139,6 @@ class DisjunctionMaxScorer extends Disju
   
   @Override
   public IntervalIterator positions(boolean collectPositions) throws IOException {
-    return new DisjunctionIntervalIterator(this, collectPositions, BooleanIntervalIterator.pullIterators(collectPositions, subScorers));
+    return new DisjunctionIntervalIterator(this, collectPositions, pullIterators(collectPositions, subScorers));
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Fri Oct 26 15:25:42 2012
@@ -20,7 +20,6 @@ package org.apache.lucene.search;
 import java.util.List;
 import java.io.IOException;
 
-import org.apache.lucene.search.positions.BooleanIntervalIterator;
 import org.apache.lucene.search.positions.ConjunctionIntervalIterator;
 import org.apache.lucene.search.positions.DisjunctionIntervalIterator;
 import org.apache.lucene.search.positions.IntervalIterator;
@@ -176,8 +175,8 @@ class DisjunctionSumScorer extends Disju
   public IntervalIterator positions(boolean collectPositions) throws IOException {
     if (minimumNrMatchers > 1) {
       return new ConjunctionIntervalIterator(this,
-          collectPositions, minimumNrMatchers, BooleanIntervalIterator.pullIterators(collectPositions, subScorers));
+          collectPositions, minimumNrMatchers, pullIterators(collectPositions, subScorers));
     }
-    return new DisjunctionIntervalIterator(this, collectPositions, BooleanIntervalIterator.pullIterators(collectPositions, subScorers));
+    return new DisjunctionIntervalIterator(this, collectPositions, pullIterators(collectPositions, subScorers));
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java Fri Oct 26 15:25:42 2012
@@ -16,7 +16,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.search.positions.BooleanIntervalIterator;
 import org.apache.lucene.search.positions.DisjunctionIntervalIterator;
 import org.apache.lucene.search.positions.IntervalIterator;
 
@@ -91,7 +90,7 @@ class ReqOptSumScorer extends Scorer {
 
   @Override
   public IntervalIterator positions(boolean collectPositions) throws IOException {
-    return new DisjunctionIntervalIterator(this, collectPositions, BooleanIntervalIterator.pullIterators(collectPositions, reqScorer, optScorer));
+    return new DisjunctionIntervalIterator(this, collectPositions, pullIterators(collectPositions, reqScorer, optScorer));
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/Scorer.java Fri Oct 26 15:25:42 2012
@@ -75,6 +75,22 @@ public abstract class Scorer extends Doc
   public abstract IntervalIterator positions(boolean collectPositions) throws IOException;
 
   /**
+   * Get the IntervalIterators from a list of scorers
+   * @param collectPositions true if positions will be collected
+   * @param scorers the list of scorers to retrieve IntervalIterators from
+   * @return a list of IntervalIterators pulled from the passed in Scorers
+   * @throws java.io.IOException
+   */
+  public static IntervalIterator[] pullIterators(boolean collectPositions, Scorer... scorers)
+      throws IOException {
+    IntervalIterator[] iterators = new IntervalIterator[scorers.length];
+    for (int i = 0; i < scorers.length; i++) {
+      iterators[i] = scorers[i].positions(collectPositions);
+    }
+    return iterators;
+  }
+
+  /**
    * Expert: Collects matching documents in a range. Hook for optimization.
    * Note, <code>firstDocID</code> is added to ensure that {@link #nextDoc()}
    * was called before this method.

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/BooleanIntervalIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/BooleanIntervalIterator.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/BooleanIntervalIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/BooleanIntervalIterator.java Fri Oct 26 15:25:42 2012
@@ -1,65 +0,0 @@
-package org.apache.lucene.search.positions;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-import org.apache.lucene.search.Scorer;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * 
- * @lucene.experimental
- */
-// nocommit - javadoc
-public abstract class BooleanIntervalIterator extends IntervalIterator {
-
-  protected final IntervalIterator[] iterators;
-  protected final IntervalQueue queue;
-
-  protected BooleanIntervalIterator(Scorer scorer, IntervalIterator[] iterators,
-      IntervalQueue queue, boolean collectPositions) throws IOException {
-    super(scorer, collectPositions);
-    this.queue = queue;
-    this.iterators = iterators;
-  }
-
-  @Override
-  public IntervalIterator[] subs(boolean inOrder) {
-    return iterators;
-  }
-
-  abstract void advance() throws IOException;
-  
-  public static IntervalIterator[] pullIterators(boolean collectPositions, Scorer... scorers)
-      throws IOException {
-    IntervalIterator[] iterators = new IntervalIterator[scorers.length];
-    for (int i = 0; i < scorers.length; i++) {
-      iterators[i] = scorers[i].positions(collectPositions);
-    }
-    return iterators;
-  }
-  
-  public static IntervalIterator[] pullIterators(boolean collectPositions,List<Scorer> scorers)
-      throws IOException {
-    IntervalIterator[] iterators = new IntervalIterator[scorers.size()];
-    for (int i = 0; i < iterators.length; i++) {
-      iterators[i] = scorers.get(i).positions(collectPositions);
-    }
-    return iterators;
-  }
-}
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionIntervalIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionIntervalIterator.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionIntervalIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/ConjunctionIntervalIterator.java Fri Oct 26 15:25:42 2012
@@ -31,11 +31,12 @@ import java.io.IOException;
  * 
  * @lucene.experimental
  */
-// nocommit - javadoc
-public final class ConjunctionIntervalIterator extends BooleanIntervalIterator {
+public final class ConjunctionIntervalIterator extends IntervalIterator {
+
   private final IntervalQueueAnd queue;
   private final int nrMustMatch;
   private SnapshotPositionCollector snapshot;
+  private final IntervalIterator[] iterators;
   private int rightExtremeBegin;
   
 
@@ -47,13 +48,13 @@ public final class ConjunctionIntervalIt
   public ConjunctionIntervalIterator(Scorer scorer, boolean collectPositions,
       int minimuNumShouldMatch, IntervalIterator... iterators)
       throws IOException {
-    super(scorer, iterators, new IntervalQueueAnd(iterators.length),
-        collectPositions);
-    this.queue = (IntervalQueueAnd) super.queue; // avoid lots of casts?
+    super(scorer, collectPositions);
+    this.iterators = iterators;
+    this.queue = new IntervalQueueAnd(iterators.length);
     this.nrMustMatch = minimuNumShouldMatch;
   }
   
-  void advance() throws IOException {
+  private void advance() throws IOException {
     final IntervalRef top = queue.top();
     Interval interval = null;
     if ((interval = iterators[top.index].next()) != null) {
@@ -115,6 +116,11 @@ public final class ConjunctionIntervalIt
     }
     return docId;
   }
+
+  @Override
+  public IntervalIterator[] subs(boolean inOrder) {
+    return iterators;
+  }
   
   
   private void snapShotSubPositions() {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionIntervalIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionIntervalIterator.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionIntervalIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/DisjunctionIntervalIterator.java Fri Oct 26 15:25:42 2012
@@ -16,11 +16,11 @@ package org.apache.lucene.search.positio
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-import java.io.IOException;
-
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.positions.IntervalQueue.IntervalRef;
 
+import java.io.IOException;
+
 /**
  * DisjunctionPositionIterator based on minimal interval semantics for OR
  * operator
@@ -31,15 +31,26 @@ import org.apache.lucene.search.position
  * 
  * @lucene.experimental
  */
-// nocommit - javadoc
-public final class DisjunctionIntervalIterator extends BooleanIntervalIterator {
+public final class DisjunctionIntervalIterator extends IntervalIterator {
+
+  private final IntervalQueue queue;
+  private final IntervalIterator[] iterators;
 
+  /**
+   * Creates a new DisjunctionIntervalIterator over a set of IntervalIterators
+   * @param scorer the parent Scorer
+   * @param collectPositions true if positions will be collected
+   * @param intervals the IntervalIterators to iterate over
+   * @throws IOException
+   */
   public DisjunctionIntervalIterator(Scorer scorer, boolean collectPositions, IntervalIterator... intervals)
       throws IOException {
-    super(scorer, intervals, new IntervalQueueOr(intervals.length), collectPositions);
+    super(scorer, collectPositions);
+    this.iterators = intervals;
+    queue = new IntervalQueueOr(intervals.length);
   }
 
-  void advance() throws IOException {
+  private void advance() throws IOException {
     final IntervalRef top = queue.top();
     Interval interval = null;
     if ((interval = iterators[top.index].next()) != null) {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/TermIntervalIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/TermIntervalIterator.java?rev=1402547&r1=1402546&r2=1402547&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/TermIntervalIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/search/positions/TermIntervalIterator.java Fri Oct 26 15:25:42 2012
@@ -16,24 +16,32 @@ package org.apache.lucene.search.positio
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.BytesRef;
 
+import java.io.IOException;
+
 
 /**
- * 
+ * Iterates over the individual positions of a term in a document
  */
-//nocommmit javadocs
 public final class TermIntervalIterator extends IntervalIterator {
+
   private final Interval interval;
   int positionsPending;
   private final DocsAndPositionsEnum docsAndPos;
   private int docID = -1;
 
-  public TermIntervalIterator(Scorer scorer, DocsAndPositionsEnum docsAndPos, boolean doPayloads,  boolean collectPositions) {
+  /**
+   * Constructs a new TermIntervalIterator
+   * @param scorer the parent Scorer
+   * @param docsAndPos a DocsAndPositionsEnum positioned on the current document
+   * @param doPayloads true if payloads should be retrieved for the positions
+   * @param collectPositions true if positions will be collected
+   */
+  public TermIntervalIterator(Scorer scorer, DocsAndPositionsEnum docsAndPos,
+                              boolean doPayloads, boolean collectPositions) {
     super(scorer, collectPositions);
     this.docsAndPos = docsAndPos;
     this.interval = doPayloads ? new PayloadInterval(docsAndPos, this)