Index: src/test/org/apache/lucene/search/payloads/TestBoostingTermQuery.java
===================================================================
--- src/test/org/apache/lucene/search/payloads/TestBoostingTermQuery.java	(revision 598017)
+++ src/test/org/apache/lucene/search/payloads/TestBoostingTermQuery.java	(working copy)
@@ -16,97 +16,42 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.analysis.*;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.Payload;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.CheckHits;
+import org.apache.lucene.search.DefaultSimilarity;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.spans.Spans;
 import org.apache.lucene.search.spans.TermSpans;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
 
 import java.io.IOException;
-import java.io.Reader;
 
 public class TestBoostingTermQuery extends LuceneTestCase {
   private IndexSearcher searcher;
   private BoostingSimilarity similarity = new BoostingSimilarity();
-  private byte[] payloadField = new byte[]{1};
-  private byte[] payloadMultiField1 = new byte[]{2};
-  private byte[] payloadMultiField2 = new byte[]{4};
 
   public TestBoostingTermQuery(String s) {
     super(s);
   }
 
-  private class PayloadAnalyzer extends Analyzer {
 
-
-    public TokenStream tokenStream(String fieldName, Reader reader) {
-      TokenStream result = new LowerCaseTokenizer(reader);
-      result = new PayloadFilter(result, fieldName);
-      return result;
-    }
-  }
-
-  private class PayloadFilter extends TokenFilter {
-    String fieldName;
-    int numSeen = 0;
-
-    public PayloadFilter(TokenStream input, String fieldName) {
-      super(input);
-      this.fieldName = fieldName;
-    }
-
-    public Token next() throws IOException {
-      Token result = input.next();
-      if (result != null) {
-        if (fieldName.equals("field")) {
-          result.setPayload(new Payload(payloadField));
-        } else if (fieldName.equals("multiField")) {
-          if (numSeen % 2 == 0) {
-            result.setPayload(new Payload(payloadMultiField1));
-          } else {
-            result.setPayload(new Payload(payloadMultiField2));
-          }
-          numSeen++;
-        }
-
-      }
-      return result;
-    }
-  }
-
   protected void setUp() throws Exception {
     super.setUp();
-    RAMDirectory directory = new RAMDirectory();
-    PayloadAnalyzer analyzer = new PayloadAnalyzer();
-    IndexWriter writer
-            = new IndexWriter(directory, analyzer, true);
-    writer.setSimilarity(similarity);
-    //writer.infoStream = System.out;
-    for (int i = 0; i < 1000; i++) {
-      Document doc = new Document();
-      Field noPayloadField = new Field("noPayLoad", English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED);
-      noPayloadField.setBoost(0);
-      doc.add(noPayloadField);
-      doc.add(new Field("field", English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
-      doc.add(new Field("multiField", English.intToEnglish(i) + "  " + English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
-      writer.addDocument(doc);
-    }
-    writer.optimize();
-    writer.close();
-
-    searcher = new IndexSearcher(directory);
-    searcher.setSimilarity(similarity);
+    PayloadHelper helper = new PayloadHelper();
+    searcher = helper.setUp(similarity, 1000);
   }
 
   public void test() throws IOException {
-    BoostingTermQuery query = new BoostingTermQuery(new Term("field", "seventy"));
+    BoostingTermQuery query = new BoostingTermQuery(new Term(PayloadHelper.FIELD, "seventy"));
     TopDocs hits = searcher.search(query, null, 100);
     assertTrue("hits is null and it shouldn't be", hits != null);
     assertTrue("hits Size: " + hits.totalHits + " is not: " + 100, hits.totalHits == 100);
@@ -119,7 +64,7 @@
       ScoreDoc doc = hits.scoreDocs[i];
       assertTrue(doc.score + " does not equal: " + 1, doc.score == 1);
     }
-    CheckHits.checkExplanations(query, "field", searcher, true);
+    CheckHits.checkExplanations(query, PayloadHelper.FIELD, searcher, true);
     Spans spans = query.getSpans(searcher.getIndexReader());
     assertTrue("spans is null and it shouldn't be", spans != null);
     assertTrue("spans is not an instanceof " + TermSpans.class, spans instanceof TermSpans);
@@ -132,7 +77,7 @@
   }
 
   public void testMultipleMatchesPerDoc() throws Exception {
-    BoostingTermQuery query = new BoostingTermQuery(new Term("multiField", "seventy"));
+    BoostingTermQuery query = new BoostingTermQuery(new Term(PayloadHelper.MULTI_FIELD, "seventy"));
     TopDocs hits = searcher.search(query, null, 100);
     assertTrue("hits is null and it shouldn't be", hits != null);
     assertTrue("hits Size: " + hits.totalHits + " is not: " + 100, hits.totalHits == 100);
@@ -169,7 +114,7 @@
   }
 
   public void testNoMatch() throws Exception {
-    BoostingTermQuery query = new BoostingTermQuery(new Term("field", "junk"));
+    BoostingTermQuery query = new BoostingTermQuery(new Term(PayloadHelper.FIELD, "junk"));
     TopDocs hits = searcher.search(query, null, 100);
     assertTrue("hits is null and it shouldn't be", hits != null);
     assertTrue("hits Size: " + hits.totalHits + " is not: " + 0, hits.totalHits == 0);
@@ -177,8 +122,8 @@
   }
 
   public void testNoPayload() throws Exception {
-    BoostingTermQuery q1 = new BoostingTermQuery(new Term("noPayLoad", "zero"));
-    BoostingTermQuery q2 = new BoostingTermQuery(new Term("noPayLoad", "foo"));
+    BoostingTermQuery q1 = new BoostingTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "zero"));
+    BoostingTermQuery q2 = new BoostingTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "foo"));
     BooleanClause c1 = new BooleanClause(q1, BooleanClause.Occur.MUST);
     BooleanClause c2 = new BooleanClause(q2, BooleanClause.Occur.MUST_NOT);
     BooleanQuery query = new BooleanQuery();
@@ -189,7 +134,7 @@
     //assertTrue("hits Size: " + hits.totalHits + " is not: " + 1, hits.totalHits == 1);
     int[] results = new int[1];
     results[0] = 0;//hits.scoreDocs[0].doc;
-    CheckHits.checkHitCollector(query, "noPayLoad", searcher, results);
+    CheckHits.checkHitCollector(query, PayloadHelper.NO_PAYLOAD_FIELD, searcher, results);
   }
 
   // must be static for weight serialization tests 
Index: src/test/org/apache/lucene/search/payloads/PayloadHelper.java
===================================================================
--- src/test/org/apache/lucene/search/payloads/PayloadHelper.java	(revision 0)
+++ src/test/org/apache/lucene/search/payloads/PayloadHelper.java	(revision 0)
@@ -0,0 +1,104 @@
+package org.apache.lucene.search.payloads;
+
+
+import org.apache.lucene.analysis.*;
+import org.apache.lucene.index.Payload;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.util.English;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Similarity;
+
+import java.io.Reader;
+import java.io.IOException;
+
+/**
+ *
+ *
+ **/
+public class PayloadHelper {
+
+  private byte[] payloadField = new byte[]{1};
+  private byte[] payloadMultiField1 = new byte[]{2};
+  private byte[] payloadMultiField2 = new byte[]{4};
+  public static final String NO_PAYLOAD_FIELD = "noPayloadField";
+  public static final String MULTI_FIELD = "multiField";
+  public static final String FIELD = "field";
+
+  public class PayloadAnalyzer extends Analyzer {
+
+
+
+    public TokenStream tokenStream(String fieldName, Reader reader) {
+      TokenStream result = new LowerCaseTokenizer(reader);
+      result = new PayloadFilter(result, fieldName);
+      return result;
+    }
+  }
+
+  public class PayloadFilter extends TokenFilter {
+    String fieldName;
+    int numSeen = 0;
+
+    public PayloadFilter(TokenStream input, String fieldName) {
+      super(input);
+      this.fieldName = fieldName;
+    }
+
+    public Token next() throws IOException {
+      Token result = input.next();
+      if (result != null) {
+        if (fieldName.equals(FIELD))
+        {
+          result.setPayload(new Payload(payloadField));
+        }
+        else if (fieldName.equals(MULTI_FIELD))
+        {
+          if (numSeen  % 2 == 0)
+          {
+            result.setPayload(new Payload(payloadMultiField1));
+          }
+          else
+          {
+            result.setPayload(new Payload(payloadMultiField2));
+          }
+          numSeen++;
+        }
+
+      }
+      return result;
+    }
+  }
+
+  /**
+   * Sets up a RAMDirectory, and adds documents (using English.intToEnglish()) with two fields: field and multiField
+   * and analyzes them using the PayloadAnalyzer
+   * @param similarity The Similarity class to use in the Searcher
+   * @param numDocs The num docs to add
+   * @return An IndexSearcher
+   * @throws IOException
+   */
+  public IndexSearcher setUp(Similarity similarity, int numDocs) throws IOException {
+    RAMDirectory directory = new RAMDirectory();
+    PayloadAnalyzer analyzer = new PayloadAnalyzer();
+    IndexWriter writer
+            = new IndexWriter(directory, analyzer, true);
+    writer.setSimilarity(similarity);
+    //writer.infoStream = System.out;
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      doc.add(new Field(FIELD, English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
+      doc.add(new Field(MULTI_FIELD, English.intToEnglish(i) + "  " + English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
+      doc.add(new Field(NO_PAYLOAD_FIELD, English.intToEnglish(i), Field.Store.YES, Field.Index.TOKENIZED));
+      writer.addDocument(doc);
+    }
+    //writer.optimize();
+    writer.close();
+
+    IndexSearcher searcher = new IndexSearcher(directory);
+    searcher.setSimilarity(similarity);
+    return searcher;
+  }
+}

Property changes on: src/test/org/apache/lucene/search/payloads/PayloadHelper.java
___________________________________________________________________
Name: svn:eol-style
   + native

Index: src/test/org/apache/lucene/search/spans/TestSpans.java
===================================================================
--- src/test/org/apache/lucene/search/spans/TestSpans.java	(revision 598017)
+++ src/test/org/apache/lucene/search/spans/TestSpans.java	(working copy)
@@ -192,6 +192,105 @@
   }
 
 
+  public void testSpanNearUnOrdered() throws Exception {
+
+    //See http://www.gossamer-threads.com/lists/lucene/java-dev/52270 for discussion about this test
+    SpanNearQuery snq;
+    snq = new SpanNearQuery(
+                              new SpanQuery[] {
+                                makeSpanTermQuery("u1"),
+                                makeSpanTermQuery("u2") },
+                              0,
+                              false);
+    Spans spans = snq.getSpans(searcher.getIndexReader());
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 4, spans.doc());
+    assertEquals("start", 1, spans.start());
+    assertEquals("end", 3, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 5, spans.doc());
+    assertEquals("start", 2, spans.start());
+    assertEquals("end", 4, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 8, spans.doc());
+    assertEquals("start", 2, spans.start());
+    assertEquals("end", 4, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 9, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 2, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 10, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 2, spans.end());
+    assertTrue("Has next and it shouldn't: " + spans.doc(), spans.next() == false);
+
+    SpanNearQuery u1u2 = new SpanNearQuery(new SpanQuery[]{makeSpanTermQuery("u1"),
+                                makeSpanTermQuery("u2")}, 0, false);
+    snq = new SpanNearQuery(
+                              new SpanQuery[] {
+                                u1u2,
+                                makeSpanTermQuery("u2")
+                              },
+                              1,
+                              false);
+    spans = snq.getSpans(searcher.getIndexReader());
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 4, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 3, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    //unordered spans can be subsets
+    assertEquals("doc", 4, spans.doc());
+    assertEquals("start", 1, spans.start());
+    assertEquals("end", 3, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 5, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 4, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 5, spans.doc());
+    assertEquals("start", 2, spans.start());
+    assertEquals("end", 4, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 8, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 4, spans.end());
+
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 8, spans.doc());
+    assertEquals("start", 2, spans.start());
+    assertEquals("end", 4, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 9, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 2, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 9, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 4, spans.end());
+
+    assertTrue("Does not have next and it should", spans.next());
+    assertEquals("doc", 10, spans.doc());
+    assertEquals("start", 0, spans.start());
+    assertEquals("end", 2, spans.end());
+
+    assertTrue("Has next and it shouldn't", spans.next() == false);
+  }
+
+
+
   private Spans orSpans(String[] terms) throws Exception {
     SpanQuery[] sqa = new SpanQuery[terms.length];
     for (int i = 0; i < terms.length; i++) {
Index: src/test/org/apache/lucene/search/spans/TestPayloadSpans.java
===================================================================
--- src/test/org/apache/lucene/search/spans/TestPayloadSpans.java	(revision 0)
+++ src/test/org/apache/lucene/search/spans/TestPayloadSpans.java	(revision 0)
@@ -0,0 +1,119 @@
+package org.apache.lucene.search.spans;
+
+/**
+ * Copyright 2004 The Apache Software Foundation
+ *
+ * Licensed 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 junit.framework.TestCase;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DefaultSimilarity;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Similarity;
+import org.apache.lucene.search.payloads.PayloadHelper;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+public class TestPayloadSpans extends TestCase {
+  private IndexSearcher searcher;
+  private Similarity similarity = new DefaultSimilarity();
+  protected IndexReader indexReader;
+
+  public TestPayloadSpans(String s) {
+    super(s);
+  }
+
+  protected void setUp() throws IOException {
+    PayloadHelper helper = new PayloadHelper();
+    searcher = helper.setUp(similarity, 1000);
+    indexReader = searcher.getIndexReader();
+  }
+
+  protected void tearDown() {
+
+  }
+
+  public void testSpanTermQuery() throws Exception {
+    SpanTermQuery stq;
+    PayloadSpans spans;
+    stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "seventy"));
+    spans = stq.getPayloadSpans(indexReader);
+    assertTrue("spans is null and it shouldn't be", spans != null);
+    checkSpans(spans, 100, 1, 1, 1);
+
+    stq = new SpanTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "seventy"));  
+    spans = stq.getPayloadSpans(indexReader);
+    assertTrue("spans is null and it shouldn't be", spans != null);
+    checkSpans(spans, 100, 0, 0, 0);
+  }
+
+  public void testSpanFirst() throws IOException {
+
+    SpanQuery match;
+    SpanFirstQuery sfq;
+    match = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
+    sfq = new SpanFirstQuery(match, 2);
+    PayloadSpans spans = sfq.getPayloadSpans(indexReader);
+    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(sfq.getPayloadSpans(indexReader), 100, 2, 1, 1);
+
+    match = new SpanNearQuery(clauses, 0, false);
+    sfq = new SpanFirstQuery(match, 2);
+    checkSpans(sfq.getPayloadSpans(indexReader), 100, 2, 1, 1);
+    
+  }
+
+  private void checkSpans(PayloadSpans 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
+    //only be one entry in the span
+    int seen = 0;
+    while (spans.next() == true)
+    {
+      //if we expect payloads, then isPayloadAvailable should be true
+      if (expectedNumPayloads > 0) {
+        assertTrue("isPayloadAvailable is not returning the correct value: " + spans.isPayloadAvailable()
+                + " and it should be: " + (expectedNumPayloads >  0),
+                spans.isPayloadAvailable() == true);
+      } else {
+        assertTrue("isPayloadAvailable should be false", spans.isPayloadAvailable() == false);
+      }
+      //See payload helper, for the PayloadHelper.FIELD field, there is a single byte payload at every token
+      if (spans.isPayloadAvailable()) {
+        Collection payload = spans.getPayload();
+        assertTrue("payload Size: " + payload.size() + " is not: " + expectedNumPayloads, payload.size() == expectedNumPayloads);
+        for (Iterator iterator = payload.iterator(); iterator.hasNext();) {
+           byte[] thePayload = (byte[]) iterator.next();
+          assertTrue("payload[0] Size: " + thePayload.length + " is not: " + expectedPayloadLength,
+                  thePayload.length == expectedPayloadLength);
+          assertTrue(thePayload[0] + " does not equal: " + expectedFirstByte, thePayload[0] == expectedFirstByte);
+
+        }
+
+      }
+      seen++;
+    }
+    assertTrue(seen + " does not equal: " + expectedNumSpans, seen == expectedNumSpans);
+  }
+}
\ No newline at end of file

Property changes on: src/test/org/apache/lucene/search/spans/TestPayloadSpans.java
___________________________________________________________________
Name: svn:eol-style
   + native

Index: src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
===================================================================
--- src/java/org/apache/lucene/search/spans/NearSpansOrdered.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/NearSpansOrdered.java	(working copy)
@@ -17,13 +17,15 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexReader;
+
 import java.io.IOException;
-
 import java.util.Arrays;
 import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Collection;
 
-import org.apache.lucene.index.IndexReader;
-
 /** A Spans that is formed from the ordered subspans of a SpanNearQuery
  * where the subspans do not overlap and have a maximum slop between them.
  * <p>
@@ -42,13 +44,13 @@
  * <pre>t1 t2 .. t3      </pre>
  * <pre>      t1 .. t2 t3</pre>
  */
-class NearSpansOrdered implements Spans {
+class NearSpansOrdered implements PayloadSpans {
   private final int allowedSlop;
   private boolean firstTime = true;
   private boolean more = false;
 
   /** The spans in the same order as the SpanNearQuery */
-  private final Spans[] subSpans;
+  private final PayloadSpans[] subSpans;
 
   /** Indicates that all subSpans have same doc() */
   private boolean inSameDoc = false;
@@ -56,8 +58,9 @@
   private int matchDoc = -1;
   private int matchStart = -1;
   private int matchEnd = -1;
+  private List/*<byte[]>*/ matchPayload;
 
-  private final Spans[] subSpansByDoc;
+  private final PayloadSpans[] subSpansByDoc;
   private final Comparator spanDocComparator = new Comparator() {
     public int compare(Object o1, Object o2) {
       return ((Spans)o1).doc() - ((Spans)o2).doc();
@@ -74,10 +77,11 @@
     }
     allowedSlop = spanNearQuery.getSlop();
     SpanQuery[] clauses = spanNearQuery.getClauses();
-    subSpans = new Spans[clauses.length];
-    subSpansByDoc = new Spans[clauses.length];
+    subSpans = new PayloadSpans[clauses.length];
+    matchPayload = new LinkedList();
+    subSpansByDoc = new PayloadSpans[clauses.length];
     for (int i = 0; i < clauses.length; i++) {
-      subSpans[i] = clauses[i].getSpans(reader);
+      subSpans[i] = clauses[i].getPayloadSpans(reader);
       subSpansByDoc[i] = subSpans[i]; // used in toSameDoc()
     }
     query = spanNearQuery; // kept for toString() only.
@@ -92,6 +96,16 @@
   // inherit javadocs
   public int end() { return matchEnd; }
 
+  // TODO: Remove warning after API has been finalized
+  public Collection/*<byte[]>*/ getPayload() throws IOException {
+    return matchPayload;
+  }
+
+  // TODO: Remove warning after API has been finalized
+ public boolean isPayloadAvailable() {
+    return matchPayload.isEmpty() == false;
+  }
+
   // inherit javadocs
   public boolean next() throws IOException {
     if (firstTime) {
@@ -104,6 +118,7 @@
       }
       more = true;
     }
+    matchPayload.clear();
     return advanceAfterOrdered();
   }
 
@@ -126,6 +141,7 @@
         return false;
       }
     }
+    matchPayload.clear();
     return advanceAfterOrdered();
   }
   
@@ -218,11 +234,14 @@
   private boolean shrinkToAfterShortestMatch() throws IOException {
     matchStart = subSpans[subSpans.length - 1].start();
     matchEnd = subSpans[subSpans.length - 1].end();
+    if (subSpans[subSpans.length - 1].isPayloadAvailable()) {
+      matchPayload.addAll(subSpans[subSpans.length - 1].getPayload());
+    }
     int matchSlop = 0;
     int lastStart = matchStart;
     int lastEnd = matchEnd;
     for (int i = subSpans.length - 2; i >= 0; i--) {
-      Spans prevSpans = subSpans[i];
+      PayloadSpans prevSpans = subSpans[i];
       int prevStart = prevSpans.start();
       int prevEnd = prevSpans.end();
       while (true) { // Advance prevSpans until after (lastStart, lastEnd)
@@ -248,6 +267,9 @@
       if (matchStart > prevEnd) { // Only non overlapping spans add to slop.
         matchSlop += (matchStart - prevEnd);
       }
+      if (subSpans[i].isPayloadAvailable()) {
+        matchPayload.addAll(0, subSpans[i].getPayload());
+      }
       /* Do not break on (matchSlop > allowedSlop) here to make sure
        * that subSpans[0] is advanced after the match, if any.
        */
Index: src/java/org/apache/lucene/search/spans/SpanFirstQuery.java
===================================================================
--- src/java/org/apache/lucene/search/spans/SpanFirstQuery.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/SpanFirstQuery.java	(working copy)
@@ -21,6 +21,7 @@
 
 import java.util.Collection;
 import java.util.Set;
+import java.util.ArrayList;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Query;
@@ -65,11 +66,15 @@
   
   public void extractTerms(Set terms) {
 	    match.extractTerms(terms);
-  }  
+  }
 
+  public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
+    return (PayloadSpans) getSpans(reader);
+  }
+
   public Spans getSpans(final IndexReader reader) throws IOException {
-    return new Spans() {
-        private Spans spans = match.getSpans(reader);
+    return new PayloadSpans() {
+        private PayloadSpans spans = match.getPayloadSpans(reader);
 
         public boolean next() throws IOException {
           while (spans.next()) {                  // scan to next match
@@ -83,17 +88,29 @@
           if (!spans.skipTo(target))
             return false;
 
-          if (spans.end() <= end)                 // there is a match
-            return true;
+          return spans.end() <= end || next();
 
-          return next();                          // scan to next match
         }
 
         public int doc() { return spans.doc(); }
         public int start() { return spans.start(); }
         public int end() { return spans.end(); }
 
-        public String toString() {
+      // TODO: Remove warning after API has been finalized
+      public Collection/*<byte[]>*/ getPayload() throws IOException {
+        ArrayList result = null;
+        if (spans.isPayloadAvailable()) {
+          result = new ArrayList(spans.getPayload());
+        }
+        return result;//TODO: any way to avoid the new construction?
+      }
+
+      // TODO: Remove warning after API has been finalized
+     public boolean isPayloadAvailable() {
+        return spans.isPayloadAvailable();
+      }
+
+      public String toString() {
           return "spans(" + SpanFirstQuery.this.toString() + ")";
         }
 
Index: src/java/org/apache/lucene/search/spans/SpanTermQuery.java
===================================================================
--- src/java/org/apache/lucene/search/spans/SpanTermQuery.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/SpanTermQuery.java	(working copy)
@@ -79,4 +79,9 @@
     return new TermSpans(reader.termPositions(term), term);
   }
 
+
+  public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
+    return (PayloadSpans) getSpans(reader);
+  }
+
 }
Index: src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
===================================================================
--- src/java/org/apache/lucene/search/spans/NearSpansUnordered.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/NearSpansUnordered.java	(working copy)
@@ -17,15 +17,17 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.PriorityQueue;
+
 import java.io.IOException;
-
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
-import java.util.ArrayList;
+import java.util.Set;
+import java.util.HashSet;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.util.PriorityQueue;
-
-class NearSpansUnordered implements Spans {
+class NearSpansUnordered implements PayloadSpans {
   private SpanNearQuery query;
 
   private List ordered = new ArrayList();         // spans in query order
@@ -60,13 +62,13 @@
 
 
   /** Wraps a Spans, and can be used to form a linked list. */
-  private class SpansCell implements Spans {
-    private Spans spans;
+  private class SpansCell implements PayloadSpans {
+    private PayloadSpans spans;
     private SpansCell next;
     private int length = -1;
     private int index;
 
-    public SpansCell(Spans spans, int index) {
+    public SpansCell(PayloadSpans spans, int index) {
       this.spans = spans;
       this.index = index;
     }
@@ -99,7 +101,16 @@
     public int doc() { return spans.doc(); }
     public int start() { return spans.start(); }
     public int end() { return spans.end(); }
+                    // TODO: Remove warning after API has been finalized
+    public Collection/*<byte[]>*/ getPayload() throws IOException {
+      return new ArrayList(spans.getPayload());
+    }
 
+    // TODO: Remove warning after API has been finalized
+   public boolean isPayloadAvailable() {
+      return spans.isPayloadAvailable();
+    }
+
     public String toString() { return spans.toString() + "#" + index; }
   }
 
@@ -113,7 +124,7 @@
     queue = new CellQueue(clauses.length);
     for (int i = 0; i < clauses.length; i++) {
       SpansCell cell =
-        new SpansCell(clauses[i].getSpans(reader), i);
+        new SpansCell(clauses[i].getPayloadSpans(reader), i);
       ordered.add(cell);
     }
   }
@@ -197,7 +208,27 @@
   public int start() { return min().start(); }
   public int end() { return max.end(); }
 
+  // TODO: Remove warning after API has been finalized
+  /**
+   * WARNING: The List is not necessarily in order of the the positions
+   * @return
+   * @throws IOException
+   */
+  public Collection/*<byte[]>*/ getPayload() throws IOException {
+    Set/*<byte[]*/ matchPayload = new HashSet();
+    for (SpansCell cell = first; cell != null; cell = cell.next) {
+      if (cell.isPayloadAvailable()) {
+        matchPayload.addAll(cell.getPayload());
+      }
+    }
+    return matchPayload;
+  }
 
+  // TODO: Remove warning after API has been finalized
+ public boolean isPayloadAvailable() {
+    return min().isPayloadAvailable();
+  }
+
   public String toString() {
     return getClass().getName() + "("+query.toString()+")@"+
       (firstTime?"START":(more?(doc()+":"+start()+"-"+end()):"END"));
@@ -214,7 +245,7 @@
     }
   }
 
-  private void addToList(SpansCell cell) {
+  private void addToList(SpansCell cell) throws IOException {
     if (last != null) {			  // add next to end of list
       last.next = cell;
     } else
@@ -230,7 +261,7 @@
     last.next = null;
   }
 
-  private void queueToList() {
+  private void queueToList() throws IOException {
     last = first = null;
     while (queue.top() != null) {
       addToList((SpansCell)queue.pop());
Index: src/java/org/apache/lucene/search/spans/SpanQuery.java
===================================================================
--- src/java/org/apache/lucene/search/spans/SpanQuery.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/SpanQuery.java	(working copy)
@@ -17,22 +17,30 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.Weight;
+
 import java.io.IOException;
-
 import java.util.Collection;
 import java.util.Set;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.Searcher;
-
 /** Base class for span-based queries. */
 public abstract class SpanQuery extends Query {
   /** Expert: Returns the matches for this query in an index.  Used internally
    * to search for spans. */
   public abstract Spans getSpans(IndexReader reader) throws IOException;
 
+  /**
+   * Returns the matches for this query in an index, including access to any {@link org.apache.lucene.index.Payload}s at those
+   * positions.
+   * @param reader
+   * @return
+   * @throws IOException
+   */
+  public abstract PayloadSpans getPayloadSpans(IndexReader reader) throws IOException;
+
   /** Returns the name of the field matched by this query.*/
   public abstract String getField();
 
Index: src/java/org/apache/lucene/search/spans/SpanNotQuery.java
===================================================================
--- src/java/org/apache/lucene/search/spans/SpanNotQuery.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/SpanNotQuery.java	(working copy)
@@ -17,15 +17,15 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.ToStringUtils;
+
 import java.io.IOException;
-
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Set;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.util.ToStringUtils;
-
 /** Removes matches which overlap with another SpanQuery. */
 public class SpanNotQuery extends SpanQuery {
   private SpanQuery include;
@@ -70,8 +70,8 @@
 
 
   public Spans getSpans(final IndexReader reader) throws IOException {
-    return new Spans() {
-        private Spans includeSpans = include.getSpans(reader);
+    return new PayloadSpans() {
+        private PayloadSpans includeSpans = include.getPayloadSpans(reader);
         private boolean moreInclude = true;
 
         private Spans excludeSpans = exclude.getSpans(reader);
@@ -131,13 +131,31 @@
         public int start() { return includeSpans.start(); }
         public int end() { return includeSpans.end(); }
 
-        public String toString() {
+      // TODO: Remove warning after API has been finalized
+      public Collection/*<byte[]>*/ getPayload() throws IOException {
+        ArrayList result = null;
+        if (includeSpans.isPayloadAvailable()) {
+          result = new ArrayList(includeSpans.getPayload());
+        }
+        return result;
+      }
+
+      // TODO: Remove warning after API has been finalized
+     public boolean isPayloadAvailable() {
+        return includeSpans.isPayloadAvailable();
+      }
+
+      public String toString() {
           return "spans(" + SpanNotQuery.this.toString() + ")";
         }
 
       };
   }
 
+  public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
+    return (PayloadSpans) getSpans(reader);
+  }
+
   public Query rewrite(IndexReader reader) throws IOException {
     SpanNotQuery clone = null;
 
Index: src/java/org/apache/lucene/search/spans/SpanOrQuery.java
===================================================================
--- src/java/org/apache/lucene/search/spans/SpanOrQuery.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/SpanOrQuery.java	(working copy)
@@ -154,19 +154,22 @@
     }
   }
 
+  public PayloadSpans getPayloadSpans(final IndexReader reader) throws IOException {
+    return (PayloadSpans)getSpans(reader);
+  }
 
   public Spans getSpans(final IndexReader reader) throws IOException {
     if (clauses.size() == 1)                      // optimize 1-clause case
-      return ((SpanQuery)clauses.get(0)).getSpans(reader);
+      return ((SpanQuery)clauses.get(0)).getPayloadSpans(reader);
 
-    return new Spans() {
+    return new PayloadSpans() {
         private SpanQueue queue = null;
 
         private boolean initSpanQueue(int target) throws IOException {
           queue = new SpanQueue(clauses.size());
           Iterator i = clauses.iterator();
           while (i.hasNext()) {
-            Spans spans = ((SpanQuery)i.next()).getSpans(reader);
+            PayloadSpans spans = ((SpanQuery)i.next()).getPayloadSpans(reader);
             if (   ((target == -1) && spans.next())
                 || ((target != -1) && spans.skipTo(target))) {
               queue.put(spans);
@@ -193,7 +196,7 @@
           return queue.size() != 0;
         }
 
-        private Spans top() { return (Spans)queue.top(); }
+        private PayloadSpans top() { return (PayloadSpans)queue.top(); }
 
         public boolean skipTo(int target) throws IOException {
           if (queue == null) {
@@ -215,7 +218,23 @@
         public int start() { return top().start(); }
         public int end() { return top().end(); }
 
-        public String toString() {
+      // TODO: Remove warning after API has been finalized
+      public Collection/*<byte[]>*/ getPayload() throws IOException {
+        ArrayList result = null;
+        PayloadSpans theTop = top();
+        if (theTop != null && theTop.isPayloadAvailable()) {
+          result = new ArrayList(theTop.getPayload());
+        }
+        return result;
+      }
+
+      // TODO: Remove warning after API has been finalized
+     public boolean isPayloadAvailable() {
+        PayloadSpans top = top();
+        return top != null && top.isPayloadAvailable();
+      }
+
+      public String toString() {
           return "spans("+SpanOrQuery.this+")@"+
             ((queue == null)?"START"
              :(queue.size()>0?(doc()+":"+start()+"-"+end()):"END"));
Index: src/java/org/apache/lucene/search/spans/TermSpans.java
===================================================================
--- src/java/org/apache/lucene/search/spans/TermSpans.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/TermSpans.java	(working copy)
@@ -20,12 +20,14 @@
 import org.apache.lucene.index.TermPositions;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.Collection;
 
 /**
  * Expert:
  * Public for extension only
  */
-public class TermSpans implements Spans {
+public class TermSpans implements PayloadSpans {
   protected TermPositions positions;
   protected Term term;
   protected int doc;
@@ -89,6 +91,18 @@
     return position + 1;
   }
 
+  // TODO: Remove warning after API has been finalized
+  public Collection/*<byte[]>*/ getPayload() throws IOException {
+    byte [] bytes = new byte[positions.getPayloadLength()]; 
+    bytes = positions.getPayload(bytes, 0);
+    return Collections.singletonList(bytes);
+  }
+
+  // TODO: Remove warning after API has been finalized
+ public boolean isPayloadAvailable() {
+    return positions.isPayloadAvailable();
+  }
+
   public String toString() {
     return "spans(" + term.toString() + ")@" +
             (doc == -1 ? "START" : (doc == Integer.MAX_VALUE) ? "END" : doc + "-" + position);
Index: src/java/org/apache/lucene/search/spans/SpanNearQuery.java
===================================================================
--- src/java/org/apache/lucene/search/spans/SpanNearQuery.java	(revision 598017)
+++ src/java/org/apache/lucene/search/spans/SpanNearQuery.java	(working copy)
@@ -120,16 +120,20 @@
 
   public Spans getSpans(final IndexReader reader) throws IOException {
     if (clauses.size() == 0)                      // optimize 0-clause case
-      return new SpanOrQuery(getClauses()).getSpans(reader);
+      return new SpanOrQuery(getClauses()).getPayloadSpans(reader);
 
     if (clauses.size() == 1)                      // optimize 1-clause case
-      return ((SpanQuery)clauses.get(0)).getSpans(reader);
+      return ((SpanQuery)clauses.get(0)).getPayloadSpans(reader);
 
     return inOrder
-            ? (Spans) new NearSpansOrdered(this, reader)
-            : (Spans) new NearSpansUnordered(this, reader);
+            ? (PayloadSpans) new NearSpansOrdered(this, reader)
+            : (PayloadSpans) new NearSpansUnordered(this, reader);
   }
 
+  public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
+    return (PayloadSpans) getSpans(reader);
+  }
+
   public Query rewrite(IndexReader reader) throws IOException {
     SpanNearQuery clone = null;
     for (int i = 0 ; i < clauses.size(); i++) {
Index: src/java/org/apache/lucene/search/spans/PayloadSpans.java
===================================================================
--- src/java/org/apache/lucene/search/spans/PayloadSpans.java	(revision 0)
+++ src/java/org/apache/lucene/search/spans/PayloadSpans.java	(revision 0)
@@ -0,0 +1,51 @@
+package org.apache.lucene.search.spans;
+
+import java.util.List;
+import java.util.Collection;
+import java.io.IOException;
+
+
+/**
+ *
+ *
+ **/
+public interface PayloadSpans extends Spans{
+   /**
+   * Returns the payload data for the current span.
+   * This is invalid until {@link #next()} is called for
+   * the first time.
+   * This method must not be called more than once after each call
+   * of {@link #next()}. However, 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>
+   * <br>
+    * Note that the return type is a collection, thus the ordering should not be relied upon.
+    * <br/>
+   * <p><font color="#FF0000">
+   * WARNING: The status of the <b>Payloads</b> feature is experimental.
+   * The APIs introduced here might change in the future and will not be
+   * supported anymore in such a case.</font>
+   *
+   * @return a List of byte arrays containing the data of this payload, otherwise null if isPayloadAvailable is false
+   * @throws java.io.IOException
+    */
+  // TODO: Remove warning after API has been finalized
+  Collection/*<byte[]>*/ getPayload() throws IOException;
+
+  /**
+   * Checks if a payload can be loaded at this position.
+   * <p/>
+   * Payloads can only be loaded once per call to
+   * {@link #next()}.
+   * <p/>
+   * <p><font color="#FF0000">
+   * WARNING: The status of the <b>Payloads</b> feature is experimental.
+   * The APIs introduced here might change in the future and will not be
+   * supported anymore in such a case.</font>
+   *
+   * @return true if there is a payload available at this position that can be loaded
+   */
+  // TODO: Remove warning after API has been finalized
+  public boolean isPayloadAvailable();
+
+}

Property changes on: src/java/org/apache/lucene/search/spans/PayloadSpans.java
___________________________________________________________________
Name: svn:eol-style
   + native

Index: contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java
===================================================================
--- contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java	(revision 598017)
+++ contrib/regex/src/java/org/apache/lucene/search/regex/SpanRegexQuery.java	(working copy)
@@ -27,6 +27,7 @@
 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.search.spans.PayloadSpans;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
@@ -76,6 +77,10 @@
     throw new UnsupportedOperationException("Query should have been rewritten");
   }
 
+  public PayloadSpans getPayloadSpans(IndexReader reader) throws IOException {
+    throw new UnsupportedOperationException("Query should have been rewritten");
+  }
+
   public String getField() {
     return term.field();
   }
