You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2015/05/22 04:15:36 UTC

svn commit: r1680978 [2/2] - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/payloads/ lucene/core/src/java/org/apache/lucene/search/spans/ lucene/core/src/test/org/apache/lucene/search/ lucene/c...

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java Fri May 22 02:15:34 2015
@@ -17,13 +17,13 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Objects;
+
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.similarities.Similarity;
 
-import java.io.IOException;
-import java.util.Objects;
-
 /**
  * Public for extension only.
  */
@@ -42,7 +42,7 @@ public class SpanScorer extends Scorer {
 
   protected SpanScorer(Spans spans, SpanWeight weight, Similarity.SimScorer docScorer) throws IOException {
     super(weight);
-    this.docScorer = docScorer;
+    this.docScorer = Objects.requireNonNull(docScorer);
     this.spans = Objects.requireNonNull(spans);
   }
 
@@ -91,10 +91,6 @@ public class SpanScorer extends Scorer {
       // assert (startPos != prevStartPos) || (endPos > prevEndPos) : "non increased endPos="+endPos;
       assert (startPos != prevStartPos) || (endPos >= prevEndPos) : "decreased endPos="+endPos;
       numMatches++;
-      if (docScorer == null) {  // scores not required, break out here
-        freq = 1;
-        return;
-      }
       int matchLength = endPos - startPos;
       freq += docScorer.computeSlopFactor(matchLength);
       prevStartPos = startPos;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Fri May 22 02:15:34 2015
@@ -17,22 +17,21 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+import java.util.Objects;
+
 import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
-import java.io.IOException;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-
 /** Matches spans containing a term.
  * This should not be used for terms that are indexed at position Integer.MAX_VALUE.
  */
@@ -51,51 +50,8 @@ public class SpanTermQuery extends SpanQ
   public String getField() { return term.field(); }
 
   @Override
-  public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
-    TermContext context = TermContext.build(searcher.getTopReaderContext(), term);
-    SpanSimilarity similarity = SpanSimilarity.build(this, searcher, needsScores, searcher.termStatistics(term, context));
-    return new SpanTermWeight(context, similarity, factory);
-  }
-
-  public class SpanTermWeight extends SpanWeight {
-
-    final TermContext termContext;
-
-    public SpanTermWeight(TermContext termContext, SpanSimilarity similarity, SpanCollectorFactory factory) throws IOException {
-      super(SpanTermQuery.this, similarity, factory);
-      this.termContext = termContext;
-    }
-
-    @Override
-    public void extractTerms(Set<Term> terms) {
-      terms.add(term);
-    }
-
-    @Override
-    public void extractTermContexts(Map<Term, TermContext> contexts) {
-      contexts.put(term, termContext);
-    }
-
-    @Override
-    public Spans getSpans(final LeafReaderContext context, Bits acceptDocs, SpanCollector collector) throws IOException {
-
-      final TermState state = termContext.get(context.ord);
-      if (state == null) { // term is not present in that reader
-        return null;
-      }
-
-      final Terms terms = context.reader().terms(term.field());
-      if (terms == null)
-        return null;
-      if (terms.hasPositions() == false)
-        throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run SpanTermQuery (term=" + term.text() + ")");
-
-      final TermsEnum termsEnum = terms.iterator();
-      termsEnum.seekExact(term.bytes(), state);
-
-      final PostingsEnum postings = termsEnum.postings(acceptDocs, null, collector.requiredPostings());
-      return new TermSpans(postings, term);
-    }
+  public void extractTerms(Set<Term> terms) {
+    terms.add(term);
   }
 
   @Override
@@ -126,4 +82,40 @@ public class SpanTermQuery extends SpanQ
     return term.equals(other.term);
   }
 
+  @Override
+  public Spans getSpans(final LeafReaderContext context, Bits acceptDocs, Map<Term,TermContext> termContexts) throws IOException {
+    TermContext termContext = termContexts.get(term);
+    final TermState state;
+    if (termContext == null) {
+      // this happens with span-not query, as it doesn't include the NOT side in extractTerms()
+      // so we seek to the term now in this segment..., this sucks because it's ugly mostly!
+      final Terms terms = context.reader().terms(term.field());
+      if (terms != null) {
+        if (terms.hasPositions() == false) {
+          throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run SpanTermQuery (term=" + term.text() + ")");
+        }
+
+        final TermsEnum termsEnum = terms.iterator();
+        if (termsEnum.seekExact(term.bytes())) {
+          state = termsEnum.termState();
+        } else {
+          state = null;
+        }
+      } else {
+        state = null;
+      }
+    } else {
+      state = termContext.get(context.ord);
+    }
+
+    if (state == null) { // term is not present in that reader
+      return null;
+    }
+
+    final TermsEnum termsEnum = context.reader().terms(term.field()).iterator();
+    termsEnum.seekExact(term.bytes(), state);
+
+    final PostingsEnum postings = termsEnum.postings(acceptDocs, null, PostingsEnum.PAYLOADS);
+    return new TermSpans(postings, term);
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java Fri May 22 02:15:34 2015
@@ -17,91 +17,88 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.Bits;
 
-import java.io.IOException;
-import java.util.Map;
-
 /**
  * Expert-only.  Public for use by other weight implementations
  */
-public abstract class SpanWeight extends Weight {
+public class SpanWeight extends Weight {
+  protected final Similarity similarity;
+  protected final Map<Term,TermContext> termContexts;
+  protected final SpanQuery query;
+  protected Similarity.SimWeight stats;
 
-  protected final SpanSimilarity similarity;
-  protected final SpanCollectorFactory collectorFactory;
-
-  /**
-   * Create a new SpanWeight
-   * @param query the parent query
-   * @param similarity a SpanSimilarity to be used for scoring
-   * @param collectorFactory a SpanCollectorFactory to be used for Span collection
-   * @throws IOException on error
-   */
-  public SpanWeight(SpanQuery query, SpanSimilarity similarity, SpanCollectorFactory collectorFactory) throws IOException {
+  public SpanWeight(SpanQuery query, IndexSearcher searcher) throws IOException {
     super(query);
-    this.similarity = similarity;
-    this.collectorFactory = collectorFactory;
+    this.similarity = searcher.getSimilarity();
+    this.query = query;
+
+    termContexts = new HashMap<>();
+    TreeSet<Term> terms = new TreeSet<>();
+    query.extractTerms(terms);
+    final IndexReaderContext context = searcher.getTopReaderContext();
+    final TermStatistics termStats[] = new TermStatistics[terms.size()];
+    int i = 0;
+    for (Term term : terms) {
+      TermContext state = TermContext.build(context, term);
+      termStats[i] = searcher.termStatistics(term, state);
+      termContexts.put(term, state);
+      i++;
+    }
+    final String field = query.getField();
+    if (field != null) {
+      stats = similarity.computeWeight(query.getBoost(),
+                                       searcher.collectionStatistics(query.getField()),
+                                       termStats);
+    }
   }
 
-  /**
-   * Collect all TermContexts used by this Weight
-   * @param contexts a map to add the TermContexts to
-   */
-  public abstract void extractTermContexts(Map<Term, TermContext> contexts);
-
-  /**
-   * Expert: Return a Spans object iterating over matches from this Weight
-   * @param ctx a LeafReaderContext for this Spans
-   * @param acceptDocs a bitset of documents to check
-   * @param collector a SpanCollector to use for postings data collection
-   * @return a Spans
-   * @throws IOException on error
-   */
-  public abstract Spans getSpans(LeafReaderContext ctx, Bits acceptDocs, SpanCollector collector) throws IOException;
-
-  /**
-   * Expert: Return a Spans object iterating over matches from this Weight, without
-   * collecting any postings data.
-   * @param ctx a LeafReaderContext for this Spans
-   * @param acceptDocs a bitset of documents to check
-   * @return a Spans
-   * @throws IOException on error
-   */
-  public final Spans getSpans(LeafReaderContext ctx, Bits acceptDocs) throws IOException {
-    return getSpans(ctx, acceptDocs, collectorFactory.newCollector());
+  @Override
+  public void extractTerms(Set<Term> terms) {
+    query.extractTerms(terms);
   }
 
   @Override
   public float getValueForNormalization() throws IOException {
-    return similarity == null ? 1.0f : similarity.getValueForNormalization();
+    return stats == null ? 1.0f : stats.getValueForNormalization();
   }
 
   @Override
   public void normalize(float queryNorm, float topLevelBoost) {
-    if (similarity != null) {
-      similarity.normalize(queryNorm, topLevelBoost);
+    if (stats != null) {
+      stats.normalize(queryNorm, topLevelBoost);
     }
   }
 
   @Override
   public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-    if (similarity == null) {
+    if (stats == null) {
       return null;
     }
-    Terms terms = context.reader().terms(similarity.getField());
+    Terms terms = context.reader().terms(query.getField());
     if (terms != null && terms.hasPositions() == false) {
-      throw new IllegalStateException("field \"" + similarity.getField() + "\" was indexed without position data; cannot run SpanQuery (query=" + parentQuery + ")");
+      throw new IllegalStateException("field \"" + query.getField() + "\" was indexed without position data; cannot run SpanQuery (query=" + query + ")");
     }
-    Spans spans = getSpans(context, acceptDocs, collectorFactory.newCollector());
-    return (spans == null) ? null : new SpanScorer(spans, this, similarity.simScorer(context));
+    Spans spans = query.getSpans(context, acceptDocs, termContexts);
+    return (spans == null) ? null : new SpanScorer(spans, this, similarity.simScorer(stats, context));
   }
 
   @Override
@@ -111,7 +108,7 @@ public abstract class SpanWeight extends
       int newDoc = scorer.advance(doc);
       if (newDoc == doc) {
         float freq = scorer.sloppyFreq();
-        SimScorer docScorer = similarity.simScorer(context);
+        SimScorer docScorer = similarity.simScorer(stats, context);
         Explanation freqExplanation = Explanation.match(freq, "phraseFreq=" + freq);
         Explanation scoreExplanation = docScorer.explain(doc, freqExplanation);
         return Explanation.match(scoreExplanation.getValue(),

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java Fri May 22 02:15:34 2015
@@ -17,16 +17,17 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.Bits;
-
 import java.io.IOException;
+import java.util.Map;
 import java.util.ArrayList;
 
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
+
 /** Keep matches that are contained within another Spans. */
 public class SpanWithinQuery extends SpanContainQuery {
-
   /** Construct a SpanWithinQuery matching spans from <code>little</code>
    * that are inside of <code>big</code>.
    * This query has the boost of <code>little</code>.
@@ -48,79 +49,62 @@ public class SpanWithinQuery extends Spa
           (SpanQuery) little.clone());
   }
 
+  /** 
+   * Return spans from <code>little</code> that are contained in a spans from <code>big</code>.
+   * The payload is from the spans of <code>little</code>.
+   */
   @Override
-  public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
-    SpanWeight bigWeight = big.createWeight(searcher, false, factory);
-    SpanWeight littleWeight = little.createWeight(searcher, false, factory);
-    SpanSimilarity similarity = SpanSimilarity.build(this, searcher, needsScores, bigWeight, littleWeight);
-    return new SpanWithinWeight(similarity, factory, bigWeight, littleWeight);
-  }
-
-  public class SpanWithinWeight extends SpanContainWeight {
-
-    public SpanWithinWeight(SpanSimilarity similarity, SpanCollectorFactory factory,
-                            SpanWeight bigWeight, SpanWeight littleWeight) throws IOException {
-      super(similarity, factory, bigWeight, littleWeight);
+  public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, final Map<Term,TermContext> termContexts) throws IOException {
+    ArrayList<Spans> containerContained = prepareConjunction(context, acceptDocs, termContexts);
+    if (containerContained == null) {
+      return null;
     }
 
-    /**
-     * Return spans from <code>little</code> that are contained in a spans from <code>big</code>.
-     * The payload is from the spans of <code>little</code>.
-     */
-    @Override
-    public Spans getSpans(final LeafReaderContext context, final Bits acceptDocs, SpanCollector collector) throws IOException {
-      ArrayList<Spans> containerContained = prepareConjunction(context, acceptDocs, collector);
-      if (containerContained == null) {
-        return null;
-      }
-
-      Spans big = containerContained.get(0);
-      Spans little = containerContained.get(1);
+    Spans big = containerContained.get(0);
+    Spans little = containerContained.get(1);
 
-      return new ContainSpans(big, little, little) {
+    return new ContainSpans(big, little, little) {
 
-        @Override
-        boolean twoPhaseCurrentDocMatches() throws IOException {
-          oneExhaustedInCurrentDoc = false;
-          assert littleSpans.startPosition() == -1;
-          while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
-            while (bigSpans.endPosition() < littleSpans.endPosition()) {
-              if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
-                oneExhaustedInCurrentDoc = true;
-                return false;
-              }
-            }
-            if (bigSpans.startPosition() <= littleSpans.startPosition()) {
-              atFirstInCurrentDoc = true;
-              return true;
+      @Override
+      boolean twoPhaseCurrentDocMatches() throws IOException {
+        oneExhaustedInCurrentDoc = false;
+        assert littleSpans.startPosition() == -1;
+        while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
+          while (bigSpans.endPosition() < littleSpans.endPosition()) {
+            if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
+              oneExhaustedInCurrentDoc = true;
+              return false;
             }
           }
-          oneExhaustedInCurrentDoc = true;
-          return false;
-        }
-
-        @Override
-        public int nextStartPosition() throws IOException {
-          if (atFirstInCurrentDoc) {
-            atFirstInCurrentDoc = false;
-            return littleSpans.startPosition();
+          if (bigSpans.startPosition() <= littleSpans.startPosition()) {
+            atFirstInCurrentDoc = true;
+            return true;
           }
-          while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
-            while (bigSpans.endPosition() < littleSpans.endPosition()) {
-              if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
-                oneExhaustedInCurrentDoc = true;
-                return NO_MORE_POSITIONS;
-              }
-            }
-            if (bigSpans.startPosition() <= littleSpans.startPosition()) {
-              return littleSpans.startPosition();
+        } 
+        oneExhaustedInCurrentDoc = true;
+        return false;
+      }
+
+      @Override
+      public int nextStartPosition() throws IOException {
+        if (atFirstInCurrentDoc) {
+          atFirstInCurrentDoc = false;
+          return littleSpans.startPosition();
+        }
+        while (littleSpans.nextStartPosition() != NO_MORE_POSITIONS) {
+          while (bigSpans.endPosition() < littleSpans.endPosition()) {
+            if (bigSpans.nextStartPosition() == NO_MORE_POSITIONS) {
+              oneExhaustedInCurrentDoc = true;
+              return NO_MORE_POSITIONS;
             }
           }
-          oneExhaustedInCurrentDoc = true;
-          return NO_MORE_POSITIONS;
+          if (bigSpans.startPosition() <= littleSpans.startPosition()) {
+            return littleSpans.startPosition();
+          }
         }
-      };
-    }
+        oneExhaustedInCurrentDoc = true;
+        return NO_MORE_POSITIONS;
+      }
+    };
   }
-
 }
\ No newline at end of file

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/Spans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/Spans.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/Spans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/Spans.java Fri May 22 02:15:34 2015
@@ -17,11 +17,12 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Collection;
+
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.TwoPhaseIterator;
 
-import java.io.IOException;
-
 /** Iterates through combinations of start/end positions per-doc.
  *  Each start/end position represents a range of term positions within the current document.
  *  These are enumerated in order, by increasing document number, within that by
@@ -50,12 +51,33 @@ public abstract class Spans extends DocI
   public abstract int endPosition();
 
   /**
-   * Collect data from the current Spans
-   * @param collector a SpanCollector
-   *
+   * Returns the payload data for the current start/end position.
+   * This is only valid after {@link #nextStartPosition()}
+   * returned an available start position.
+   * This method must not be called more than once after each call
+   * of {@link #nextStartPosition()}. However, most payloads are loaded lazily,
+   * so if the payload data for the current position is not needed,
+   * this method may not be called at all for performance reasons.
+   * <br>
+   * Note that the return type is a collection, thus the ordering should not be relied upon.
+   * <br>
    * @lucene.experimental
+   *
+   * @return a List of byte arrays containing the data of this payload, otherwise null if isPayloadAvailable is false
+   * @throws IOException if there is a low-level I/O error
+   */
+  public abstract Collection<byte[]> getPayload() throws IOException;
+
+  /**
+   * Checks if a payload can be loaded at the current start/end position.
+   * <p>
+   * Payloads can only be loaded once per call to
+   * {@link #nextStartPosition()}.
+   *
+   * @return true if there is a payload available at this start/end position
+   *              that can be loaded
    */
-  public abstract void collect(SpanCollector collector) throws IOException;
+  public abstract boolean isPayloadAvailable() throws IOException;
 
   /**
    * Optional method: Return a {@link TwoPhaseIterator} view of this

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java Fri May 22 02:15:34 2015
@@ -16,11 +16,14 @@ package org.apache.lucene.search.spans;
  */
 
 
-import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.Collection;
 import java.util.Objects;
 
 /**
@@ -106,7 +109,6 @@ public class TermSpans extends Spans {
     return postings.cost();
   }
 
-  /*
   @Override
   public Collection<byte[]> getPayload() throws IOException {
     final BytesRef payload = postings.getPayload();
@@ -125,12 +127,6 @@ public class TermSpans extends Spans {
   public boolean isPayloadAvailable() throws IOException {
     return readPayload == false && postings.getPayload() != null;
   }
-  */
-
-  @Override
-  public void collect(SpanCollector collector) throws IOException {
-    collector.collectLeaf(postings, term);
-  }
 
   @Override
   public String toString() {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestPositionIncrement.java Fri May 22 02:15:34 2015
@@ -17,37 +17,34 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.MockPayloadAnalyzer;
-import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import java.io.IOException;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+
+import org.apache.lucene.analysis.*;
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.payloads.PayloadSpanCollector;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.search.payloads.PayloadSpanUtil;
 import org.apache.lucene.search.spans.MultiSpansWrapper;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.search.spans.Spans;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.nio.charset.StandardCharsets;
-import java.util.Collection;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * Term position unit test.
@@ -56,7 +53,7 @@ import java.util.Collection;
  */
 public class TestPositionIncrement extends LuceneTestCase {
 
-  final static boolean VERBOSE = true;
+  final static boolean VERBOSE = false;
 
   public void testSetPosition() throws Exception {
     Analyzer analyzer = new Analyzer() {
@@ -241,17 +238,14 @@ public class TestPositionIncrement exten
     if (VERBOSE) {
       System.out.println("\ngetPayloadSpans test");
     }
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    Spans pspans = MultiSpansWrapper.wrap(is.getIndexReader(), snq, collector);
+    Spans pspans = MultiSpansWrapper.wrap(is.getIndexReader(), snq);
     while (pspans.nextDoc() != Spans.NO_MORE_DOCS) {
       while (pspans.nextStartPosition() != Spans.NO_MORE_POSITIONS) {
         if (VERBOSE) {
           System.out.println("doc " + pspans.docID() + ": span " + pspans.startPosition()
               + " to " + pspans.endPosition());
         }
-        collector.reset();
-        pspans.collect(collector);
-        Collection<byte[]> payloads = collector.getPayloads();
+        Collection<byte[]> payloads = pspans.getPayload();
         sawZero |= pspans.startPosition() == 0;
         for (byte[] bytes : payloads) {
           count++;
@@ -262,7 +256,7 @@ public class TestPositionIncrement exten
       }
     }
     assertTrue(sawZero);
-    assertEquals(8, count);
+    assertEquals(5, count);
 
     // System.out.println("\ngetSpans test");
     Spans spans = MultiSpansWrapper.wrap(is.getIndexReader(), snq);
@@ -288,7 +282,7 @@ public class TestPositionIncrement exten
       //System.out.println(s);
       sawZero |= s.equals("pos: 0");
     }
-    assertEquals(8, count);
+    assertEquals(5, count);
     assertTrue(sawZero);
     writer.close();
     is.getIndexReader().close();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadBasics.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadBasics.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadBasics.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadBasics.java Fri May 22 02:15:34 2015
@@ -17,6 +17,13 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.SimplePayloadFilter;
@@ -29,7 +36,9 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.CheckHits;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
+import org.apache.lucene.search.spans.SpanNearPayloadCheckQuery;
 import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanPayloadCheckQuery;
 import org.apache.lucene.search.spans.SpanPositionRangeQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
@@ -41,13 +50,6 @@ import org.apache.lucene.util.TestUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
 /** basic test of payload-spans */
 public class TestPayloadBasics extends LuceneTestCase {
   private static IndexSearcher searcher;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java Fri May 22 02:15:34 2015
@@ -15,12 +15,9 @@ package org.apache.lucene.search.payload
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import java.io.IOException;
 
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.analysis.TokenFilter;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.*;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -30,14 +27,14 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.QueryUtils;
 import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
-import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
@@ -46,7 +43,6 @@ import org.apache.lucene.util.LuceneTest
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import java.io.IOException;
 
 public class TestPayloadNearQuery extends LuceneTestCase {
   private static IndexSearcher searcher;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadSpans.java Fri May 22 02:15:34 2015
@@ -16,23 +16,32 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.payloads.PayloadHelper;
+import org.apache.lucene.search.payloads.PayloadSpanUtil;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.spans.MultiSpansWrapper;
@@ -46,13 +55,6 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 
-import java.io.IOException;
-import java.io.StringReader;
-import java.nio.charset.StandardCharsets;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
 public class TestPayloadSpans extends LuceneTestCase {
   private IndexSearcher searcher;
   private Similarity similarity = new DefaultSimilarity();
@@ -72,15 +74,14 @@ public class TestPayloadSpans extends Lu
     SpanTermQuery stq;
     Spans spans;
     stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "seventy"));
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    spans = MultiSpansWrapper.wrap(indexReader, stq, collector);
+    spans = MultiSpansWrapper.wrap(indexReader, stq);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    checkSpans(spans, collector, 100, 1, 1, 1);
+    checkSpans(spans, 100, 1, 1, 1);
 
     stq = new SpanTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "seventy"));  
-    spans = MultiSpansWrapper.wrap(indexReader, stq, collector);
+    spans = MultiSpansWrapper.wrap(indexReader, stq);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    checkSpans(spans, collector, 100, 0, 0, 0);
+    checkSpans(spans, 100, 0, 0, 0);
   }
 
   public void testSpanFirst() throws IOException {
@@ -89,20 +90,19 @@ public class TestPayloadSpans extends Lu
     SpanFirstQuery sfq;
     match = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
     sfq = new SpanFirstQuery(match, 2);
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    Spans spans = MultiSpansWrapper.wrap(indexReader, sfq, collector);
-    checkSpans(spans, collector, 109, 1, 1, 1);
+    Spans spans = MultiSpansWrapper.wrap(indexReader, sfq);
+    checkSpans(spans, 109, 1, 1, 1);
     //Test more complicated subclause
     SpanQuery[] clauses = new SpanQuery[2];
     clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
     clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "hundred"));
     match = new SpanNearQuery(clauses, 0, true);
     sfq = new SpanFirstQuery(match, 2);
-    checkSpans(MultiSpansWrapper.wrap(indexReader, sfq, collector), collector, 100, 2, 1, 1);
+    checkSpans(MultiSpansWrapper.wrap(indexReader, sfq), 100, 2, 1, 1);
 
     match = new SpanNearQuery(clauses, 0, false);
     sfq = new SpanFirstQuery(match, 2);
-    checkSpans(MultiSpansWrapper.wrap(indexReader, sfq, collector), collector, 100, 2, 1, 1);
+    checkSpans(MultiSpansWrapper.wrap(indexReader, sfq), 100, 2, 1, 1);
     
   }
   
@@ -124,9 +124,9 @@ public class TestPayloadSpans extends Lu
     writer.addDocument(doc);
     IndexReader reader = writer.getReader();
     writer.close();
+    
 
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    checkSpans(MultiSpansWrapper.wrap(reader, snq, collector), collector, 1, new int[]{2});
+    checkSpans(MultiSpansWrapper.wrap(reader, snq), 1,new int[]{2});
     reader.close();
     directory.close();
   }
@@ -135,10 +135,8 @@ public class TestPayloadSpans extends Lu
     SpanTermQuery stq;
     Spans spans;
     IndexSearcher searcher = getSearcher();
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-
     stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "mark"));
-    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), stq, collector);
+    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), stq);
     assertNull(spans);
 
     SpanQuery[] clauses = new SpanQuery[3];
@@ -147,9 +145,9 @@ public class TestPayloadSpans extends Lu
     clauses[2] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "xx"));
     SpanNearQuery spanNearQuery = new SpanNearQuery(clauses, 12, false);
 
-    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), spanNearQuery, collector);
+    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), spanNearQuery);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    checkSpans(spans, collector, 2, new int[]{3,3});
+    checkSpans(spans, 2, new int[]{3,3});
 
      
     clauses[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "xx"));
@@ -158,10 +156,10 @@ public class TestPayloadSpans extends Lu
 
     spanNearQuery = new SpanNearQuery(clauses, 6, true);
    
-    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), spanNearQuery, collector);
+    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), spanNearQuery);
 
     assertTrue("spans is null and it shouldn't be", spans != null);
-    checkSpans(spans, collector, 1, new int[]{3});
+    checkSpans(spans, 1, new int[]{3});
      
     clauses = new SpanQuery[2];
      
@@ -180,9 +178,10 @@ public class TestPayloadSpans extends Lu
     SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses2, 6, false);
     
     // yy within 6 of xx within 6 of rr
-    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), nestedSpanNearQuery, collector);
+
+    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), nestedSpanNearQuery);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    checkSpans(spans, collector, 2, new int[]{3,3});
+    checkSpans(spans, 2, new int[]{3,3});
     closeIndexReader.close();
     directory.close();
   }
@@ -209,13 +208,12 @@ public class TestPayloadSpans extends Lu
      
     clauses3[0] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "np"));
     clauses3[1] = snq;
-
-    PayloadSpanCollector collector = new PayloadSpanCollector();
+     
     SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses3, 6, false);
-    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), nestedSpanNearQuery, collector);
+    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), nestedSpanNearQuery);
 
     assertTrue("spans is null and it shouldn't be", spans != null);
-    checkSpans(spans, collector, 1, new int[]{3});
+    checkSpans(spans, 1, new int[]{3});
     closeIndexReader.close();
     directory.close();
   }
@@ -250,10 +248,9 @@ public class TestPayloadSpans extends Lu
      
     SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses3, 6, false);
 
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), nestedSpanNearQuery, collector);
+    spans = MultiSpansWrapper.wrap(searcher.getIndexReader(), nestedSpanNearQuery);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    checkSpans(spans, collector, 2, new int[]{8, 8});
+    checkSpans(spans, 2, new int[]{8, 8});
     closeIndexReader.close();
     directory.close();
   }
@@ -275,17 +272,15 @@ public class TestPayloadSpans extends Lu
     SpanTermQuery stq2 = new SpanTermQuery(new Term("content", "k"));
     SpanQuery[] sqs = { stq1, stq2 };
     SpanNearQuery snq = new SpanNearQuery(sqs, 1, true);
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    Spans spans = MultiSpansWrapper.wrap(is.getIndexReader(), snq, collector);
+    Spans spans = MultiSpansWrapper.wrap(is.getIndexReader(), snq);
 
     TopDocs topDocs = is.search(snq, 1);
     Set<String> payloadSet = new HashSet<>();
     for (int i = 0; i < topDocs.scoreDocs.length; i++) {
       while (spans.nextDoc() != Spans.NO_MORE_DOCS) {
         while (spans.nextStartPosition() != Spans.NO_MORE_POSITIONS) {
-          collector.reset();
-          spans.collect(collector);
-          Collection<byte[]> payloads = collector.getPayloads();
+          Collection<byte[]> payloads = spans.getPayload();
+  
           for (final byte [] payload : payloads) {
             payloadSet.add(new String(payload, StandardCharsets.UTF_8));
           }
@@ -315,17 +310,14 @@ public class TestPayloadSpans extends Lu
     SpanTermQuery stq2 = new SpanTermQuery(new Term("content", "k"));
     SpanQuery[] sqs = { stq1, stq2 };
     SpanNearQuery snq = new SpanNearQuery(sqs, 0, true);
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    Spans spans =  MultiSpansWrapper.wrap(is.getIndexReader(), snq, collector);
+    Spans spans =  MultiSpansWrapper.wrap(is.getIndexReader(), snq);
 
     TopDocs topDocs = is.search(snq, 1);
     Set<String> payloadSet = new HashSet<>();
     for (int i = 0; i < topDocs.scoreDocs.length; i++) {
       while (spans.nextDoc() != Spans.NO_MORE_DOCS) {
         while (spans.nextStartPosition() != Spans.NO_MORE_POSITIONS) {
-          collector.reset();
-          spans.collect(collector);
-          Collection<byte[]> payloads = collector.getPayloads();
+          Collection<byte[]> payloads = spans.getPayload();
   
           for (final byte [] payload : payloads) {
             payloadSet.add(new String(payload, StandardCharsets.UTF_8));
@@ -356,17 +348,14 @@ public class TestPayloadSpans extends Lu
     SpanTermQuery stq2 = new SpanTermQuery(new Term("content", "k"));
     SpanQuery[] sqs = { stq1, stq2 };
     SpanNearQuery snq = new SpanNearQuery(sqs, 0, true);
-    PayloadSpanCollector collector = new PayloadSpanCollector();
-    Spans spans =  MultiSpansWrapper.wrap(is.getIndexReader(), snq, collector);
+    Spans spans =  MultiSpansWrapper.wrap(is.getIndexReader(), snq);
 
     TopDocs topDocs = is.search(snq, 1);
     Set<String> payloadSet = new HashSet<>();
     for (int i = 0; i < topDocs.scoreDocs.length; i++) {
       while (spans.nextDoc() != Spans.NO_MORE_DOCS) {
         while (spans.nextStartPosition() != Spans.NO_MORE_POSITIONS) {
-          collector.reset();
-          spans.collect(collector);
-          Collection<byte[]> payloads = collector.getPayloads();
+          Collection<byte[]> payloads = spans.getPayload();
   
           for (final byte [] payload : payloads) {
             payloadSet.add(new String(payload, StandardCharsets.UTF_8));
@@ -412,7 +401,7 @@ public class TestPayloadSpans extends Lu
     directory.close();
   }
 
-  private void checkSpans(Spans spans, PayloadSpanCollector collector, int expectedNumSpans, int expectedNumPayloads,
+  private void checkSpans(Spans spans, int expectedNumSpans, int expectedNumPayloads,
                           int expectedPayloadLength, int expectedFirstByte) throws IOException {
     assertTrue("spans is null and it shouldn't be", spans != null);
     //each position match should have a span associated with it, since there is just one underlying term query, there should
@@ -420,16 +409,16 @@ public class TestPayloadSpans extends Lu
     int seen = 0;
     while (spans.nextDoc() != Spans.NO_MORE_DOCS) {
       while (spans.nextStartPosition() != Spans.NO_MORE_POSITIONS) {
-        collector.reset();
-        spans.collect(collector);
-
-        Collection<byte[]> payload = collector.getPayloads();
-        assertEquals("payload size", expectedNumPayloads, payload.size());
-        for (final byte [] thePayload : payload) {
-          assertEquals("payload length", expectedPayloadLength, thePayload.length);
-          assertEquals("payload first byte", expectedFirstByte, thePayload[0]);
+        assertEquals("isPayloadAvailable should return true/false as payloads are expected", expectedNumPayloads > 0, spans.isPayloadAvailable());
+        //See payload helper, for the PayloadHelper.FIELD field, there is a single byte payload at every token
+        if (spans.isPayloadAvailable()) {
+          Collection<byte[]> payload = spans.getPayload();
+          assertEquals("payload size", expectedNumPayloads, payload.size());
+          for (final byte [] thePayload : payload) {
+            assertEquals("payload length", expectedPayloadLength, thePayload.length);
+            assertEquals("payload first byte", expectedFirstByte, thePayload[0]);
+          }
         }
-
         seen++;
       }
     }
@@ -457,26 +446,26 @@ public class TestPayloadSpans extends Lu
     return searcher;
   }
   
-  private void checkSpans(Spans spans, PayloadSpanCollector collector, int numSpans, int[] numPayloads) throws IOException {
+  private void checkSpans(Spans spans, int numSpans, int[] numPayloads) throws IOException {
     int cnt = 0;
 
     while (spans.nextDoc() != Spans.NO_MORE_DOCS) {
       while (spans.nextStartPosition() != Spans.NO_MORE_POSITIONS) {
         if(VERBOSE)
           System.out.println("\nSpans Dump --");
-        collector.reset();
-        spans.collect(collector);
-
-        Collection<byte[]> payload = collector.getPayloads();
-        if(VERBOSE) {
-          System.out.println("payloads for span:" + payload.size());
-          for (final byte [] bytes : payload) {
-            System.out.println("doc:" + spans.docID() + " s:" + spans.startPosition() + " e:" + spans.endPosition() + " "
+        if (spans.isPayloadAvailable()) {
+          Collection<byte[]> payload = spans.getPayload();
+          if(VERBOSE) {
+            System.out.println("payloads for span:" + payload.size());
+            for (final byte [] bytes : payload) {
+              System.out.println("doc:" + spans.docID() + " s:" + spans.startPosition() + " e:" + spans.endPosition() + " "
                 + new String(bytes, StandardCharsets.UTF_8));
+            }
           }
+          assertEquals("payload size", numPayloads[cnt], payload.size());
+        } else { // no payload available
+          assertFalse("Expected spans:" + numPayloads[cnt] + " found: 0", numPayloads.length > 0 && numPayloads[cnt] > 0 );
         }
-        assertEquals("payload size", numPayloads[cnt], payload.size());
-
         cnt++;
       }
     }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java Fri May 22 02:15:34 2015
@@ -16,35 +16,31 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.analysis.TokenFilter;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.analysis.*;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.English;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.QueryUtils;
-import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.spans.MultiSpansWrapper;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.search.spans.Spans;
+import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.English;
-import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java Fri May 22 02:15:34 2015
@@ -17,10 +17,16 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.similarities.Similarity;
-
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
+import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.Bits;
 
 /**
  * Holds all implementations of classes in the o.a.l.s.spans package as a
@@ -59,14 +65,19 @@ final class JustCompileSearchSpans {
     public int endPosition() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
-
+    
     @Override
-    public void collect(SpanCollector collector) throws IOException {
+    public int nextStartPosition() throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
 
+    @Override
+    public Collection<byte[]> getPayload() {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
     @Override
-    public int nextStartPosition() throws IOException {
+    public boolean isPayloadAvailable() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
@@ -79,12 +90,17 @@ final class JustCompileSearchSpans {
   static final class JustCompileSpanQuery extends SpanQuery {
 
     @Override
+    protected void extractTerms(Set<Term> terms) {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
+
+    @Override
     public String getField() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
     @Override
-    public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
+    public Spans getSpans(LeafReaderContext context, Bits acceptDocs, Map<Term,TermContext> termContexts) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
@@ -121,14 +137,19 @@ final class JustCompileSearchSpans {
     public int endPosition() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
-
+    
     @Override
-    public void collect(SpanCollector collector) throws IOException {
+    public int nextStartPosition() throws IOException {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
 
+    @Override
+    public Collection<byte[]> getPayload() {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
     @Override
-    public int nextStartPosition() throws IOException {
+    public boolean isPayloadAvailable() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/MultiSpansWrapper.java Fri May 22 02:15:34 2015
@@ -17,15 +17,19 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
 import org.apache.lucene.util.Bits;
 
-import java.io.IOException;
-
 /**
  * 
  * A wrapper to perform span operations on a non-leaf reader context
@@ -36,18 +40,17 @@ import java.io.IOException;
 public class MultiSpansWrapper {
 
   public static Spans wrap(IndexReader reader, SpanQuery spanQuery) throws IOException {
-    return wrap(reader, spanQuery, SpanCollector.NO_OP);
-  }
-
-  public static Spans wrap(IndexReader reader, SpanQuery spanQuery, SpanCollector collector) throws IOException {
-
     LeafReader lr = SlowCompositeReaderWrapper.wrap(reader); // slow, but ok for testing
     LeafReaderContext lrContext = lr.getContext();
-    IndexSearcher searcher = new IndexSearcher(lr);
-    searcher.setQueryCache(null);
-
-    SpanWeight w = (SpanWeight) searcher.createNormalizedWeight(spanQuery, false);
-
-    return w.getSpans(lrContext, new Bits.MatchAllBits(lr.numDocs()), collector);
+    SpanQuery rewrittenQuery = (SpanQuery) spanQuery.rewrite(lr); // get the term contexts so getSpans can be called directly
+    HashSet<Term> termSet = new HashSet<>();
+    rewrittenQuery.extractTerms(termSet);
+    Map<Term,TermContext> termContexts = new HashMap<>();
+    for (Term term: termSet) {
+      TermContext termContext = TermContext.build(lrContext, term);
+      termContexts.put(term, termContext);
+    }
+    Spans actSpans = spanQuery.getSpans(lrContext, new Bits.MatchAllBits(lr.numDocs()), termContexts);
+    return actSpans;
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java Fri May 22 02:15:34 2015
@@ -17,6 +17,9 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.util.HashSet;
+import java.util.Set;
+
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
@@ -33,11 +36,7 @@ import org.apache.lucene.util.LuceneTest
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import java.util.HashSet;
-import java.util.Set;
-
-import static org.apache.lucene.search.spans.SpanTestUtil.assertFinished;
-import static org.apache.lucene.search.spans.SpanTestUtil.assertNext;
+import static org.apache.lucene.search.spans.SpanTestUtil.*;
 
 public class TestFieldMaskingSpanQuery extends LuceneTestCase {
 
@@ -142,7 +141,7 @@ public class TestFieldMaskingSpanQuery e
     QueryUtils.checkEqual(q, qr);
 
     Set<Term> terms = new HashSet<>();
-    qr.createWeight(searcher, false).extractTerms(terms);
+    qr.extractTerms(terms);
     assertEquals(1, terms.size());
   }
   
@@ -162,7 +161,7 @@ public class TestFieldMaskingSpanQuery e
     QueryUtils.checkUnequal(q, qr);
 
     Set<Term> terms = new HashSet<>();
-    qr.createWeight(searcher, false).extractTerms(terms);
+    qr.extractTerms(terms);
     assertEquals(2, terms.size());
   }
   

Modified: lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java (original)
+++ lucene/dev/branches/branch_5x/lucene/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java Fri May 22 02:15:34 2015
@@ -17,6 +17,17 @@ package org.apache.lucene.search.highlig
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
 import org.apache.lucene.analysis.CachingTokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.index.BinaryDocValues;
@@ -29,6 +40,7 @@ import org.apache.lucene.index.LeafReade
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.queries.CommonTermsQuery;
@@ -53,21 +65,10 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanOrQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
-import org.apache.lucene.search.spans.SpanWeight;
 import org.apache.lucene.search.spans.Spans;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
 
 /**
  * Class used to extract {@link WeightedSpanTerm}s from a {@link Query} based on whether 
@@ -299,9 +300,14 @@ public class WeightedSpanTermExtractor {
         q = spanQuery;
       }
       LeafReaderContext context = getLeafContext();
-      SpanWeight w = (SpanWeight) searcher.createNormalizedWeight(q, false);
+      Map<Term,TermContext> termContexts = new HashMap<>();
+      TreeSet<Term> extractedTerms = new TreeSet<>();
+      searcher.createNormalizedWeight(q, false).extractTerms(extractedTerms);
+      for (Term term : extractedTerms) {
+        termContexts.put(term, TermContext.build(context, term));
+      }
       Bits acceptDocs = context.reader().getLiveDocs();
-      final Spans spans = w.getSpans(context, acceptDocs);
+      final Spans spans = q.getSpans(context, acceptDocs, termContexts);
       if (spans == null) {
         return;
       }

Modified: lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java (original)
+++ lucene/dev/branches/branch_5x/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterTest.java Fri May 22 02:15:34 2015
@@ -84,7 +84,7 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanNotQuery;
 import org.apache.lucene.search.spans.SpanOrQuery;
-import org.apache.lucene.search.payloads.SpanPayloadCheckQuery;
+import org.apache.lucene.search.spans.SpanPayloadCheckQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpanQuery.java Fri May 22 02:15:34 2015
@@ -17,11 +17,17 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.TermContext;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
-
-import java.io.IOException;
+import org.apache.lucene.util.Bits;
 
 /** Wraps a span query with asserts */
 public class AssertingSpanQuery extends SpanQuery {
@@ -32,6 +38,21 @@ public class AssertingSpanQuery extends
   }
 
   @Override
+  protected void extractTerms(Set<Term> terms) {
+    in.extractTerms(terms);
+  }
+
+  @Override
+  public Spans getSpans(LeafReaderContext context, Bits acceptDocs, Map<Term,TermContext> termContexts) throws IOException {
+    Spans spans = in.getSpans(context, acceptDocs, termContexts);
+    if (spans == null) {
+      return null;
+    } else {
+      return new AssertingSpans(spans);
+    }
+  }
+
+  @Override
   public String getField() {
     return in.getField();
   }
@@ -42,9 +63,15 @@ public class AssertingSpanQuery extends
   }
 
   @Override
-  public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, SpanCollectorFactory factory) throws IOException {
-    SpanWeight weight = in.createWeight(searcher, needsScores, factory);
-    return new AssertingSpanWeight(weight);
+  public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    // TODO: we are wasteful and createWeight twice in this case... use VirtualMethod?
+    // we need to not wrap if the query is e.g. a Payload one that overrides this (it should really be final)
+    SpanWeight weight = in.createWeight(searcher, needsScores);
+    if (weight.getClass() == SpanWeight.class) {
+      return super.createWeight(searcher, needsScores);
+    } else {
+      return weight;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpans.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/spans/AssertingSpans.java Fri May 22 02:15:34 2015
@@ -17,10 +17,12 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Collection;
+
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.TwoPhaseIterator;
-
-import java.io.IOException;
+import org.apache.lucene.search.spans.Spans;
 
 /** 
  * Wraps a Spans with additional asserts 
@@ -123,13 +125,19 @@ class AssertingSpans extends Spans {
     checkCurrentPositions();
     return in.endPosition();
   }
-
+  
   @Override
-  public void collect(SpanCollector collector) throws IOException {
-    assert state == State.ITERATING : "collect() called in illegal state: " + state + ": " + in;
-    in.collect(collector);
+  public Collection<byte[]> getPayload() throws IOException {
+    assert state == State.ITERATING : "getPayload() called in illegal state: " + state + ": " + in;
+    return in.getPayload();
   }
-
+  
+  @Override
+  public boolean isPayloadAvailable() throws IOException {
+    assert state == State.ITERATING : "isPayloadAvailable() called in illegal state: " + state + ": " + in;
+    return in.isPayloadAvailable();
+  }
+  
   @Override
   public int docID() {
     int doc = in.docID();

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java?rev=1680978&r1=1680977&r2=1680978&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/highlight/HighlighterTest.java Fri May 22 02:15:34 2015
@@ -28,7 +28,7 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.payloads.SpanPayloadCheckQuery;
+import org.apache.lucene.search.spans.SpanPayloadCheckQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.params.HighlightParams;