You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2014/11/12 15:56:24 UTC

svn commit: r1638805 [7/8] - in /lucene/dev/lucene2878: dev-tools/idea/lucene/highlighter/ lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/ lucene/analysis/common/src/test/org/apache/lucene/analysis/standard/ lucene/codecs/src/java/org...

Modified: lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java (original)
+++ lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/postingshighlight/PostingsHighlighter.java Wed Nov 12 14:56:17 2014
@@ -17,21 +17,8 @@ package org.apache.lucene.search.posting
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.text.BreakIterator;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.SortedSet;
-import java.util.TreeSet;
-
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
@@ -53,6 +40,19 @@ import org.apache.lucene.util.InPlaceMer
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 
+import java.io.IOException;
+import java.text.BreakIterator;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
 /**
  * Simple highlighter that does not analyze fields nor use
  * term vectors. Instead it requires 
@@ -455,7 +455,7 @@ public class PostingsHighlighter {
     
   private Map<Integer,Object> highlightField(String field, String contents[], BreakIterator bi, BytesRef terms[], int[] docids, List<LeafReaderContext> leaves, int maxPassages, Query query) throws IOException {  
     Map<Integer,Object> highlights = new HashMap<>();
-    
+
     PassageFormatter fieldFormatter = getFormatter(field);
     if (fieldFormatter == null) {
       throw new NullPointerException("PassageFormatter cannot be null");
@@ -477,7 +477,7 @@ public class PostingsHighlighter {
 
     // we are processing in increasing docid order, so we only need to reinitialize stuff on segment changes
     // otherwise, we will just advance() existing enums to the new document in the same segment.
-    DocsAndPositionsEnum postings[] = null;
+    DocsEnum postings[] = null;
     TermsEnum termsEnum = null;
     int lastLeaf = -1;
     
@@ -499,7 +499,7 @@ public class PostingsHighlighter {
         Terms t = r.terms(field);
         if (t != null) {
           termsEnum = t.iterator(null);
-          postings = new DocsAndPositionsEnum[terms.length];
+          postings = new DocsEnum[terms.length];
         }
       }
       if (termsEnum == null) {
@@ -508,7 +508,7 @@ public class PostingsHighlighter {
       
       // if there are multi-term matches, we have to initialize the "fake" enum for each document
       if (automata.length > 0) {
-        DocsAndPositionsEnum dp = MultiTermHighlighting.getDocsEnum(analyzer.tokenStream(field, content), automata);
+        DocsEnum dp = MultiTermHighlighting.getDocsEnum(analyzer.tokenStream(field, content), automata);
         dp.advance(doc - subContext.docBase);
         postings[terms.length-1] = dp; // last term is the multiterm matcher
       }
@@ -534,7 +534,7 @@ public class PostingsHighlighter {
   // we can intersect these with the postings lists via BreakIterator.preceding(offset),s
   // score each sentence as norm(sentenceStartOffset) * sum(weight * tf(freq))
   private Passage[] highlightDoc(String field, BytesRef terms[], int contentLength, BreakIterator bi, int doc, 
-      TermsEnum termsEnum, DocsAndPositionsEnum[] postings, int n) throws IOException {
+      TermsEnum termsEnum, DocsEnum[] postings, int n) throws IOException {
     PassageScorer scorer = getScorer(field);
     if (scorer == null) {
       throw new NullPointerException("PassageScorer cannot be null");
@@ -543,7 +543,7 @@ public class PostingsHighlighter {
     float weights[] = new float[terms.length];
     // initialize postings
     for (int i = 0; i < terms.length; i++) {
-      DocsAndPositionsEnum de = postings[i];
+      DocsEnum de = postings[i];
       int pDoc;
       if (de == EMPTY) {
         continue;
@@ -552,7 +552,7 @@ public class PostingsHighlighter {
         if (!termsEnum.seekExact(terms[i])) {
           continue; // term not found
         }
-        de = postings[i] = termsEnum.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_OFFSETS);
+        de = postings[i] = termsEnum.docsAndPositions(null, null, DocsEnum.FLAG_OFFSETS);
         if (de == null) {
           // no positions available
           throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
@@ -590,7 +590,7 @@ public class PostingsHighlighter {
     
     OffsetsEnum off;
     while ((off = pq.poll()) != null) {
-      final DocsAndPositionsEnum dp = off.dp;
+      final DocsEnum dp = off.dp;
       int start = dp.startOffset();
       if (start == -1) {
         throw new IllegalArgumentException("field '" + field + "' was indexed without offsets, cannot highlight");
@@ -698,11 +698,11 @@ public class PostingsHighlighter {
   }
   
   private static class OffsetsEnum implements Comparable<OffsetsEnum> {
-    DocsAndPositionsEnum dp;
+    DocsEnum dp;
     int pos;
     int id;
     
-    OffsetsEnum(DocsAndPositionsEnum dp, int id) throws IOException {
+    OffsetsEnum(DocsEnum dp, int id) throws IOException {
       this.dp = dp;
       this.id = id;
       this.pos = 1;
@@ -724,10 +724,10 @@ public class PostingsHighlighter {
     }
   }
   
-  private static final DocsAndPositionsEnum EMPTY = new DocsAndPositionsEnum() {
+  private static final DocsEnum EMPTY = new DocsEnum() {
 
     @Override
-    public int nextPosition() throws IOException { return 0; }
+    public int nextPosition() throws IOException { return NO_MORE_POSITIONS; }
 
     @Override
     public int startOffset() throws IOException { return Integer.MAX_VALUE; }

Modified: lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java (original)
+++ lucene/dev/lucene2878/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java Wed Nov 12 14:56:17 2014
@@ -22,7 +22,7 @@ import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.Set;
 
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
@@ -93,7 +93,7 @@ public class FieldTermStack {
 
     final CharsRefBuilder spare = new CharsRefBuilder();
     final TermsEnum termsEnum = vector.iterator(null);
-    DocsAndPositionsEnum dpEnum = null;
+    DocsEnum dpEnum = null;
     BytesRef text;
     
     int numDocs = reader.maxDoc();

Modified: lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java (original)
+++ lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/custom/HighlightCustomQueryTest.java Wed Nov 12 14:56:17 2014
@@ -16,9 +16,6 @@ package org.apache.lucene.search.highlig
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-import java.io.IOException;
-import java.util.Map;
-
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
 import org.apache.lucene.analysis.MockTokenizer;
@@ -36,6 +33,9 @@ import org.apache.lucene.search.highligh
 import org.apache.lucene.search.highlight.WeightedSpanTermExtractor;
 import org.apache.lucene.util.LuceneTestCase;
 
+import java.io.IOException;
+import java.util.Map;
+
 /**
  * Tests the extensibility of {@link WeightedSpanTermExtractor} and
  * {@link QueryScorer} in a user defined package

Added: lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java?rev=1638805&view=auto
==============================================================================
--- lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java (added)
+++ lucene/dev/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/positions/IntervalHighlighterTest.java Wed Nov 12 14:56:17 2014
@@ -0,0 +1,512 @@
+package org.apache.lucene.search.highlight.positions;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenFilter;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MultiPhraseQuery;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.WildcardQuery;
+import org.apache.lucene.search.highlight.Highlighter;
+import org.apache.lucene.search.highlight.InvalidTokenOffsetsException;
+import org.apache.lucene.search.highlight.SimpleFragmenter;
+import org.apache.lucene.search.highlight.TextFragment;
+import org.apache.lucene.search.posfilter.NonOverlappingQuery;
+import org.apache.lucene.search.posfilter.OrderedNearQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.junit.Ignore;
+
+import java.io.IOException;
+import java.io.StringReader;
+
+/**
+ * TODO: FIX THIS TEST Phrase and Span Queries positions callback API
+ */
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
+public class IntervalHighlighterTest extends LuceneTestCase {
+  
+  protected final static String F = "f";
+  protected Analyzer analyzer;
+  protected Directory dir;
+  protected IndexSearcher searcher;
+  private IndexWriterConfig iwc;
+  
+  private static final String PORRIDGE_VERSE = "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some"
+      + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!";
+  
+  public void setUp() throws Exception {
+    super.setUp();
+    iwc = newIndexWriterConfig(new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)).setOpenMode(OpenMode.CREATE);
+    analyzer = iwc.getAnalyzer();
+    dir = newDirectory();
+  }
+  
+  public void close() throws IOException {
+    if (searcher != null) {
+      searcher.getIndexReader().close();
+      searcher = null;
+    }
+    dir.close();
+  }
+  
+  // make several docs
+  protected void insertDocs(Analyzer analyzer, String... values)
+      throws Exception {
+    IndexWriter writer = new IndexWriter(dir, iwc);
+    FieldType type = new FieldType();
+    type.setIndexed(true);
+    type.setTokenized(true);
+    type.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    type.setStored(true);
+    for (String value : values) {
+      Document doc = new Document();
+      Field f = newField(F, value, type);
+      doc.add(f);
+      writer.addDocument(doc);
+    }
+    writer.close();
+    if (searcher != null) {
+      searcher.getIndexReader().close();
+    }
+    searcher = new IndexSearcher(DirectoryReader.open(dir));
+  }
+
+  protected static TermQuery termQuery(String term) {
+    return new TermQuery(new Term(F, term));
+  }
+  
+  private String[] doSearch(Query q) throws IOException,
+      InvalidTokenOffsetsException {
+    return doSearch(q, 100);
+  }
+  
+  private class ConstantScorer implements
+      org.apache.lucene.search.highlight.Scorer {
+    
+    @Override
+    public TokenStream init(TokenStream tokenStream) throws IOException {
+      return tokenStream;
+    }
+    
+    @Override
+    public void startFragment(TextFragment newFragment) {}
+    
+    @Override
+    public float getTokenScore() {
+      return 1;
+    }
+    
+    @Override
+    public float getFragmentScore() {
+      return 1;
+    }
+  }
+
+  private String getHighlight(Query q) throws IOException, InvalidTokenOffsetsException {
+    return doSearch(q, Integer.MAX_VALUE)[0];
+  }
+  
+  private String[] doSearch(Query q, int maxFragSize) throws IOException,
+      InvalidTokenOffsetsException {
+    return doSearch(q, maxFragSize, 0);
+  }
+  private String[] doSearch(Query q, int maxFragSize, int docIndex) throws IOException, InvalidTokenOffsetsException {
+    return doSearch(q, maxFragSize, docIndex, false);
+  }
+  private String[] doSearch(Query q, int maxFragSize, int docIndex, boolean analyze)
+      throws IOException, InvalidTokenOffsetsException {
+    // ConstantScorer is a fragment Scorer, not a search result (document)
+    // Scorer
+    Highlighter highlighter = new Highlighter(new ConstantScorer());
+    highlighter.setTextFragmenter(new SimpleFragmenter(maxFragSize));
+    HighlightingIntervalCollector collector = new HighlightingIntervalCollector(10);
+    if (q instanceof MultiTermQuery) {
+      ((MultiTermQuery) q)
+          .setRewriteMethod(MultiTermQuery.CONSTANT_SCORE_BOOLEAN_QUERY_REWRITE);
+    }
+    searcher.search(q, collector);
+    DocAndPositions doc = collector.docs[docIndex];
+    if (doc == null) return null;
+    String text = searcher.getIndexReader().document(doc.doc).get(F);
+    // FIXME: test error cases: for non-stored fields, and fields w/no term
+    // vectors
+    // searcher.getIndexReader().getTermFreqVector(doc.doc, F, pom);
+    final TokenStream stream;
+    if (analyze) {
+      stream = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true,
+          MockTokenFilter.EMPTY_STOPSET).tokenStream(F,
+          new StringReader(text));
+    } else {
+      stream = new IntervalTokenStream(text, doc.sortedPositions());
+    }
+    //
+    TextFragment[] fragTexts = highlighter.getBestTextFragments(
+         stream , text, false, 10);
+    String[] frags = new String[fragTexts.length];
+    for (int i = 0; i < frags.length; i++)
+      frags[i] = fragTexts[i].toString();
+    return frags;
+  }
+  
+  public void testTerm() throws Exception {
+    insertDocs(analyzer, "This is a test test");
+    String frags[] = doSearch(termQuery("test"));
+    assertEquals("This is a <B>test</B> <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testSeveralSnippets() throws Exception {
+    String input = "this is some long text.  It has the word long in many places.  In fact, it has long on some different fragments.  "
+        + "Let us see what happens to long in this case.";
+    String gold = "this is some <B>long</B> text.  It has the word <B>long</B> in many places.  In fact, it has <B>long</B> on some different fragments.  "
+        + "Let us see what happens to <B>long</B> in this case.";
+    insertDocs(analyzer, input);
+    String frags[] = doSearch(termQuery("long"), input.length());
+    assertEquals(gold, frags[0]);
+    close();
+  }
+  
+  public void testBooleanAnd() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(termQuery("This"), Occur.MUST));
+    bq.add(new BooleanClause(termQuery("test"), Occur.MUST));
+    String frags[] = doSearch(bq);
+    assertEquals("<B>This</B> is a <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testConstantScore() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(termQuery("This"), Occur.MUST));
+    bq.add(new BooleanClause(termQuery("test"), Occur.MUST));
+    String frags[] = doSearch(new ConstantScoreQuery(bq));
+    assertEquals("<B>This</B> is a <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testBooleanAndOtherOrder() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.MUST));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "This")), Occur.MUST));
+    String frags[] = doSearch(bq);
+    assertEquals("<B>This</B> is a <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testBooleanOr() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "This")), Occur.SHOULD));
+    String frags[] = doSearch(bq);
+    assertEquals("<B>This</B> is a <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testSingleMatchScorer() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "notoccurringterm")),
+        Occur.SHOULD));
+    String frags[] = doSearch(bq);
+    assertEquals("This is a <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testBooleanNrShouldMatch() throws Exception {
+    insertDocs(analyzer, "a b c d e f g h i");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "a")), Occur.SHOULD));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "b")), Occur.SHOULD));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "no")), Occur.SHOULD));
+    
+    // This generates a ConjunctionSumScorer
+    bq.setMinimumNumberShouldMatch(2);
+    String frags[] = doSearch(bq);
+    assertEquals("<B>a</B> <B>b</B> c d e f g h i", frags[0]);
+    
+    // This generates no scorer
+    bq.setMinimumNumberShouldMatch(3);
+    frags = doSearch(bq);
+    assertNull(frags);
+    
+    // This generates a DisjunctionSumScorer
+    bq.setMinimumNumberShouldMatch(2);
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "c")), Occur.SHOULD));
+    frags = doSearch(bq);
+    assertEquals("<B>a</B> <B>b</B> <B>c</B> d e f g h i", frags[0]);
+    close();
+  }
+  
+  public void testPhrase() throws Exception {
+    insertDocs(analyzer, "is it that this is a test, is it");
+    PhraseQuery pq = new PhraseQuery();
+    pq.add(new Term(F, "is"));
+    pq.add(new Term(F, "a"));
+    String frags[] = doSearch(pq);
+    // make sure we highlight the phrase, and not the terms outside the phrase
+    assertEquals("is it that this <B>is</B> <B>a</B> test, is it", frags[0]);
+    close();
+  }
+  
+  /*
+   * Failing ... PhraseQuery scorer needs positions()?
+   */
+  //@Ignore
+  public void testPhraseOriginal() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    PhraseQuery pq = new PhraseQuery();
+    pq.add(new Term(F, "a"));
+    pq.add(new Term(F, "test"));
+    String frags[] = doSearch(pq);
+    assertEquals("This is <B>a</B> <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testNestedBoolean() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
+    BooleanQuery bq2 = new BooleanQuery();
+    bq2.add(new BooleanClause(new TermQuery(new Term(F, "This")), Occur.SHOULD));
+    bq2.add(new BooleanClause(new TermQuery(new Term(F, "is")), Occur.SHOULD));
+    bq.add(new BooleanClause(bq2, Occur.SHOULD));
+    String frags[] = doSearch(bq);
+    assertEquals("<B>This</B> <B>is</B> a <B>test</B>", frags[0]);
+    close();
+  }
+  
+  public void testWildcard() throws Exception {
+    insertDocs(analyzer, "This is a test");
+    String frags[] = doSearch(new WildcardQuery(new Term(F, "t*t")));
+    assertEquals("This is a <B>test</B>", frags[0]);
+    close();
+  }
+
+  public void testMixedBooleanNot() throws Exception {
+    insertDocs(analyzer, "this is a test", "that is an elephant");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.MUST));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "that")), Occur.MUST_NOT));
+    String frags[] = doSearch(bq);
+    assertEquals("this is a <B>test</B>", frags[0]);
+    close();
+  }
+
+  public void testMixedBooleanShould() throws Exception {
+    insertDocs(analyzer, "this is a test", "that is an elephant", "the other was a rhinoceros");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "is")), Occur.MUST));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "test")), Occur.SHOULD));
+    String frags[] = doSearch(bq, 50, 0);
+    assertEquals("this <B>is</B> a <B>test</B>", frags[0]);
+    frags = doSearch(bq, 50, 1);
+    assertEquals("that <B>is</B> an elephant", frags[0]);
+
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "rhinoceros")), Occur.SHOULD));
+    frags = doSearch(bq, 50, 0);
+    assertEquals("this <B>is</B> a <B>test</B>", frags[0]);
+    frags = doSearch(bq, 50, 1);
+    assertEquals("that <B>is</B> an elephant", frags[0]);
+    close();
+  }
+  
+  public void testMultipleDocumentsAnd() throws Exception {
+    insertDocs(analyzer, "This document has no matches", PORRIDGE_VERSE,
+        "This document has some Pease porridge in it");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "Pease")), Occur.MUST));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "porridge")), Occur.MUST));
+    String frags[] = doSearch(bq, 50, 0);
+    assertEquals(
+        "<B>Pease</B> <B>porridge</B> hot! <B>Pease</B> <B>porridge</B> cold! <B>Pease</B>",
+        frags[0]);
+    frags = doSearch(bq, 50, 1);
+    assertEquals("This document has some <B>Pease</B> <B>porridge</B> in it",
+        frags[0]);
+    close();
+  }
+  
+
+  public void testMultipleDocumentsOr() throws Exception {
+    insertDocs(analyzer, "This document has no matches", PORRIDGE_VERSE,
+        "This document has some Pease porridge in it");
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "Pease")), Occur.SHOULD));
+    bq.add(new BooleanClause(new TermQuery(new Term(F, "porridge")),
+        Occur.SHOULD));
+    String frags[] = doSearch(bq, 50, 0);
+    assertEquals(
+        "<B>Pease</B> <B>porridge</B> hot! <B>Pease</B> <B>porridge</B> cold! <B>Pease</B>",
+        frags[0]);
+    frags = doSearch(bq, 50, 1);
+    assertEquals("This document has some <B>Pease</B> <B>porridge</B> in it",
+        frags[0]);
+    close();
+  }
+  
+  public void testBrouwerianQuery() throws Exception {
+
+    insertDocs(analyzer, "the quick brown duck jumps over the lazy dog with the quick brown fox");
+
+    BooleanQuery query = new BooleanQuery();
+    query.add(new BooleanClause(new TermQuery(new Term(F, "the")), Occur.MUST));
+    query.add(new BooleanClause(new TermQuery(new Term(F, "quick")), Occur.MUST));
+    query.add(new BooleanClause(new TermQuery(new Term(F, "jumps")), Occur.MUST));
+
+    assertEquals(getHighlight(query),
+                 "<B>the</B> <B>quick</B> brown duck <B>jumps</B> over <B>the</B> lazy dog with the <B>quick</B> brown fox");
+
+    BooleanQuery sub = new BooleanQuery();
+    sub.add(new BooleanClause(new TermQuery(new Term(F, "duck")), Occur.MUST));
+    NonOverlappingQuery bq = new NonOverlappingQuery(query, sub);
+
+    assertEquals(getHighlight(bq),
+                 "the quick brown duck <B>jumps</B> over <B>the</B> lazy dog with the <B>quick</B> brown fox");
+
+    close();
+  }
+  
+  @Ignore("not implemented yet - unsupported")
+  public void testMultiPhraseQuery() throws Exception {
+    MultiPhraseQuery query = new MultiPhraseQuery();
+    insertDocs(analyzer, "pease porridge hot but not too hot or otherwise pease porridge cold");
+
+    query.add(terms(F, "pease"), 0);
+    query.add(terms(F, "porridge"), 1);
+    query.add(terms(F, "hot", "cold"), 2);
+    query.setSlop(1);
+    
+    String[] frags = doSearch(query, Integer.MAX_VALUE);
+    assertEquals("<B>pease</B> <B>porridge</B> <B>hot</B> but not too hot or otherwise <B>pease</B> <B>porridge</B> <B>cold</B>", frags[0]);
+
+    close();
+  }
+  
+  @Ignore("not implemented yet - unsupported")
+  public void testMultiPhraseQueryCollisions() throws Exception {
+    MultiPhraseQuery query = new MultiPhraseQuery();
+    insertDocs(analyzer, "pease porridge hot not too hot or otherwise pease porridge porridge");
+
+    query.add(terms(F, "pease"), 0);
+    query.add(terms(F, "porridge"), 1);
+    query.add(terms(F, "coldasice", "porridge" ), 2);
+    query.setSlop(1);
+    
+    String[] frags = doSearch(query, Integer.MAX_VALUE);
+    assertEquals("pease porridge hot but not too hot or otherwise <B>pease</B> <B>porridge</B> <B>porridge</B>", frags[0]);
+
+    close();
+  }
+
+  public void testNearPhraseQuery() throws Exception {
+
+    insertDocs(analyzer, "pease porridge rather hot and pease porridge fairly cold");
+
+    Query firstQ = new OrderedNearQuery(4, termQuery("pease"), termQuery("porridge"), termQuery("hot"));
+    {
+      String frags[] = doSearch(firstQ, Integer.MAX_VALUE);
+      assertEquals("<B>pease</B> <B>porridge</B> rather <B>hot</B> and pease porridge fairly cold", frags[0]);
+    }
+
+    // near.3(near.4(pease, porridge, hot), near.4(pease, porridge, cold))
+    Query q = new OrderedNearQuery(3,
+                firstQ,
+                new OrderedNearQuery(4, termQuery("pease"), termQuery("porridge"), termQuery("cold")));
+
+    String frags[] = doSearch(q, Integer.MAX_VALUE);
+    assertEquals("<B>pease</B> <B>porridge</B> rather <B>hot</B> and <B>pease</B> <B>porridge</B> fairly <B>cold</B>",
+                 frags[0]);
+
+    close();
+  }
+
+  private Term[] terms(String field, String...tokens) {
+      Term[] terms = new Term[tokens.length];
+      for (int i = 0; i < tokens.length; i++) {
+        terms[i] = new Term(field, tokens[i]);
+      }
+      return terms;
+    }
+
+  public void testSloppyPhraseQuery() throws Exception {
+    assertSloppyPhrase( "a b c d a b c d e f", "a b <B>c</B> d <B>a</B> b c d e f", 2, "c", "a");
+    assertSloppyPhrase( "a c e b d e f a b","<B>a</B> c e <B>b</B> d e f <B>a</B> <B>b</B>", 2, "a", "b");
+    assertSloppyPhrase( "Y A X B A", "Y <B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
+
+    assertSloppyPhrase( "X A X B A","<B>X</B> <B>A</B> X B <B>A</B>", 2, "X", "A", "A"); // non overlapping minmal!!
+    assertSloppyPhrase( "A A A X",null, 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X A",  "A <B>A</B> <B>X</B> <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X A Y B A", "A <B>A</B> <B>X</B> <B>A</B> Y B <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X", null, 2, "X", "A", "A");
+    assertSloppyPhrase( "A X A", null, 1, "X", "A", "A");
+
+    assertSloppyPhrase( "A X B A", "<B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A <B>A</B> <B>X</B> <B>A</B> <B>X</B> B <B>A</B> <B>X</B> B B <B>A</B> <B>A</B> <B>X</B> B <B>A</B> <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X A X B A X B B A A X B A A", "A <B>A</B> <B>X</B> <B>A</B> <B>X</B> B <B>A</B> <B>X</B> B B <B>A</B> <B>A</B> <B>X</B> B <B>A</B> <B>A</B>", 2, "X", "A", "A");
+
+    assertSloppyPhrase( "A A X A X B A", "A <B>A</B> <B>X</B> <B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A Y A X B A", "A A Y <B>A</B> <B>X</B> B <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A Y A X B A A", "A A Y <B>A</B> <B>X</B> B <B>A</B> <B>A</B>", 2, "X", "A", "A");
+    assertSloppyPhrase( "A A X A Y B A", null , 1, "X", "A", "A");
+    close();
+  }
+
+
+  private void assertSloppyPhrase(String doc, String expected, int slop, String...query) throws Exception {
+    insertDocs(analyzer, doc);
+    PhraseQuery pq = new PhraseQuery();
+    for (String string : query) {
+      pq.add(new Term(F, string));  
+    }
+    
+    pq.setSlop(slop);
+//    System.out.println(doc);
+    String[] frags = doSearch(pq, 50);
+    if (expected == null) {
+      assertNull(frags != null ? frags[0] : "", frags);
+    } else {
+      assertEquals(expected, frags[0]);
+    }
+  }
+  
+}

Modified: lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java (original)
+++ lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java Wed Nov 12 14:56:17 2014
@@ -17,12 +17,13 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
-import java.util.Collection;
-
 import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 
+import java.io.IOException;
+import java.util.Collection;
+
 /** Passed to {@link LeafCollector#setScorer} during join collection. */
 final class FakeScorer extends Scorer {
   float score;
@@ -49,6 +50,11 @@ final class FakeScorer extends Scorer {
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public int nextDoc() {
     throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
   }

Modified: lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (original)
+++ lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java Wed Nov 12 14:56:17 2014
@@ -17,10 +17,6 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Locale;
-import java.util.Set;
-
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -42,6 +38,10 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.FixedBitSet;
 
+import java.io.IOException;
+import java.util.Locale;
+import java.util.Set;
+
 class TermsIncludingScoreQuery extends Query {
 
   final String field;
@@ -133,7 +133,7 @@ class TermsIncludingScoreQuery extends Q
 
       @Override
       public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-        SVInnerScorer scorer = (SVInnerScorer) bulkScorer(context, false, null);
+        SVInnerScorer scorer = (SVInnerScorer) bulkScorer(context, false, DocsEnum.FLAG_FREQS, null);
         if (scorer != null) {
           return scorer.explain(doc);
         }
@@ -163,7 +163,7 @@ class TermsIncludingScoreQuery extends Q
       }
 
       @Override
-      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      public Scorer scorer(LeafReaderContext context, int flags, Bits acceptDocs) throws IOException {
         Terms terms = context.reader().terms(field);
         if (terms == null) {
           return null;
@@ -181,10 +181,10 @@ class TermsIncludingScoreQuery extends Q
       }
 
       @Override
-      public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+      public BulkScorer bulkScorer(LeafReaderContext context, boolean scoreDocsInOrder, int flags, Bits acceptDocs) throws IOException {
 
         if (scoreDocsInOrder) {
-          return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
+          return super.bulkScorer(context, scoreDocsInOrder, flags, acceptDocs);
         } else {
           Terms terms = context.reader().terms(field);
           if (terms == null) {
@@ -288,6 +288,7 @@ class TermsIncludingScoreQuery extends Q
 
       return new ComplexExplanation(true, scores[ords[scoreUpto]], "Score based on join value " + termsEnum.term().utf8ToString());
     }
+
   }
 
   // This impl that tracks whether a docid has already been emitted. This check makes sure that docs aren't emitted
@@ -361,6 +362,11 @@ class TermsIncludingScoreQuery extends Q
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return -1;
+    }
+
+    @Override
     public int docID() {
       return currentDoc;
     }

Modified: lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java (original)
+++ lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java Wed Nov 12 14:56:17 2014
@@ -35,6 +35,11 @@ import org.apache.lucene.util.BitDocIdSe
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+
 /**
  * Just like {@link ToParentBlockJoinQuery}, except this
  * query joins in reverse: you provide a Query matching
@@ -122,9 +127,9 @@ public class ToChildBlockJoinQuery exten
     // NOTE: acceptDocs applies (and is checked) only in the
     // child document space
     @Override
-    public Scorer scorer(LeafReaderContext readerContext, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext readerContext, int flags, Bits acceptDocs) throws IOException {
 
-      final Scorer parentScorer = parentWeight.scorer(readerContext, null);
+      final Scorer parentScorer = parentWeight.scorer(readerContext, flags, null);
 
       if (parentScorer == null) {
         // No matches
@@ -273,6 +278,11 @@ public class ToChildBlockJoinQuery exten
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return -1; // nocommit do positions make sense here?
+    }
+
+    @Override
     public int advance(int childTarget) throws IOException {
       assert childTarget >= parentBits.length() || !parentBits.get(childTarget);
       

Modified: lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java (original)
+++ lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java Wed Nov 12 14:56:17 2014
@@ -17,16 +17,32 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.FieldValueHitQueue;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreCachingWrappingScorer;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.IndexWriter; // javadocs
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.Scorer.ChildScorer;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TopDocsCollector;
+import org.apache.lucene.search.TopFieldCollector;
+import org.apache.lucene.search.TopScoreDocCollector;
 import org.apache.lucene.search.grouping.GroupDocs;
 import org.apache.lucene.search.grouping.TopGroups;
 import org.apache.lucene.util.ArrayUtil;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Queue;
 
 
 /** Collects parent document hits for a Query containing one more more

Modified: lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java (original)
+++ lucene/dev/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java Wed Nov 12 14:56:17 2014
@@ -17,12 +17,7 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Locale;
-import java.util.Set;
-
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.LeafReaderContext;
@@ -40,6 +35,12 @@ import org.apache.lucene.util.BitDocIdSe
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Locale;
+import java.util.Set;
+
 /**
  * This query requires that you index
  * children and parent docs as a single block, using the
@@ -155,9 +156,9 @@ public class ToParentBlockJoinQuery exte
     // NOTE: acceptDocs applies (and is checked) only in the
     // parent document space
     @Override
-    public Scorer scorer(LeafReaderContext readerContext, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext readerContext, int flags, Bits acceptDocs) throws IOException {
 
-      final Scorer childScorer = childWeight.scorer(readerContext, readerContext.reader().getLiveDocs());
+      final Scorer childScorer = childWeight.scorer(readerContext, flags, readerContext.reader().getLiveDocs());
       if (childScorer == null) {
         // No matches
         return null;
@@ -183,7 +184,7 @@ public class ToParentBlockJoinQuery exte
 
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      BlockJoinScorer scorer = (BlockJoinScorer) scorer(context, context.reader().getLiveDocs());
+      BlockJoinScorer scorer = (BlockJoinScorer) scorer(context, DocsEnum.FLAG_FREQS, context.reader().getLiveDocs());
       if (scorer != null && scorer.advance(doc) == doc) {
         return scorer.explain(context.docBase);
       }
@@ -370,6 +371,11 @@ public class ToParentBlockJoinQuery exte
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return -1; // nocommit do positions make sense here?
+    }
+
+    @Override
     public int advance(int parentTarget) throws IOException {
 
       //System.out.println("Q.advance parentTarget=" + parentTarget);

Modified: lucene/dev/lucene2878/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java (original)
+++ lucene/dev/lucene2878/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java Wed Nov 12 14:56:17 2014
@@ -17,6 +17,57 @@ package org.apache.lucene.search.join;
  * limitations under the License.
  */
 
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LogDocMergePolicy;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.NoMergePolicy;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.index.StoredDocument;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.FieldDoc;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryUtils;
+import org.apache.lucene.search.QueryWrapperFilter;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.grouping.GroupDocs;
+import org.apache.lucene.search.grouping.TopGroups;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
+import org.apache.lucene.util.TestUtil;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -24,16 +75,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
 
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.*;
-import org.apache.lucene.index.*;
-import org.apache.lucene.search.*;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.grouping.GroupDocs;
-import org.apache.lucene.search.grouping.TopGroups;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.*;
-
 public class TestBlockJoin extends LuceneTestCase {
 
   // One resume...
@@ -1148,7 +1189,7 @@ public class TestBlockJoin extends Lucen
 
     ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
     Weight weight = s.createNormalizedWeight(q);
-    DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
+    DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), DocsEnum.FLAG_FREQS, null);
     assertEquals(1, disi.advance(1));
     r.close();
     dir.close();
@@ -1182,7 +1223,7 @@ public class TestBlockJoin extends Lucen
 
     ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
     Weight weight = s.createNormalizedWeight(q);
-    DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
+    DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), DocsEnum.FLAG_FREQS, null);
     assertEquals(2, disi.advance(0));
     r.close();
     dir.close();

Modified: lucene/dev/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Wed Nov 12 14:56:17 2014
@@ -17,15 +17,6 @@ package org.apache.lucene.index.memory;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.NoSuchElementException;
-
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -34,7 +25,6 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
@@ -61,16 +51,24 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.BytesRefHash.DirectBytesStartArray;
 import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.IntBlockPool.SliceReader;
 import org.apache.lucene.util.IntBlockPool.SliceWriter;
-import org.apache.lucene.util.IntBlockPool;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.RecyclingByteBlockAllocator;
 import org.apache.lucene.util.RecyclingIntBlockAllocator;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
 
 /**
  * High-performance single-document main memory Apache Lucene fulltext search index. 
@@ -1015,19 +1013,16 @@ public class MemoryIndex {
 
       @Override
       public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
-        if (reuse == null || !(reuse instanceof MemoryDocsEnum)) {
-          reuse = new MemoryDocsEnum();
-        }
-        return ((MemoryDocsEnum) reuse).reset(liveDocs, info.sliceArray.freq[info.sortedTerms[termUpto]]);
+        return docsAndPositions(liveDocs, reuse, flags);
       }
 
       @Override
-      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
-        if (reuse == null || !(reuse instanceof MemoryDocsAndPositionsEnum)) {
-          reuse = new MemoryDocsAndPositionsEnum();
+      public DocsEnum docsAndPositions(Bits liveDocs, DocsEnum reuse, int flags) {
+        if (reuse == null || !(reuse instanceof MemoryDocsEnum)) {
+          reuse = new MemoryDocsEnum();
         }
         final int ord = info.sortedTerms[termUpto];
-        return ((MemoryDocsAndPositionsEnum) reuse).reset(liveDocs, info.sliceArray.start[ord], info.sliceArray.end[ord], info.sliceArray.freq[ord]);
+        return ((MemoryDocsEnum) reuse).reset(liveDocs, info.sliceArray.start[ord], info.sliceArray.end[ord], info.sliceArray.freq[ord]);
       }
 
       @Override
@@ -1045,51 +1040,6 @@ public class MemoryIndex {
     }
     
     private class MemoryDocsEnum extends DocsEnum {
-      private boolean hasNext;
-      private Bits liveDocs;
-      private int doc = -1;
-      private int freq;
-
-      public DocsEnum reset(Bits liveDocs, int freq) {
-        this.liveDocs = liveDocs;
-        hasNext = true;
-        doc = -1;
-        this.freq = freq;
-        return this;
-      }
-
-      @Override
-      public int docID() {
-        return doc;
-      }
-
-      @Override
-      public int nextDoc() {
-        if (hasNext && (liveDocs == null || liveDocs.get(0))) {
-          hasNext = false;
-          return doc = 0;
-        } else {
-          return doc = NO_MORE_DOCS;
-        }
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        return slowAdvance(target);
-      }
-
-      @Override
-      public int freq() throws IOException {
-        return freq;
-      }
-
-      @Override
-      public long cost() {
-        return 1;
-      }
-    }
-    
-    private class MemoryDocsAndPositionsEnum extends DocsAndPositionsEnum {
       private int posUpto; // for assert
       private boolean hasNext;
       private Bits liveDocs;
@@ -1099,11 +1049,11 @@ public class MemoryIndex {
       private int startOffset;
       private int endOffset;
       
-      public MemoryDocsAndPositionsEnum() {
+      public MemoryDocsEnum() {
         this.sliceReader = new SliceReader(intBlockPool);
       }
 
-      public DocsAndPositionsEnum reset(Bits liveDocs, int start, int end, int freq) {
+      public DocsEnum reset(Bits liveDocs, int start, int end, int freq) {
         this.liveDocs = liveDocs;
         this.sliceReader.reset(start, end);
         posUpto = 0; // for assert
@@ -1141,7 +1091,9 @@ public class MemoryIndex {
 
       @Override
       public int nextPosition() {
-        assert posUpto++ < freq;
+        //assert posUpto++ < freq;
+        if (posUpto++ >= freq)
+          return NO_MORE_POSITIONS;
         assert !sliceReader.endOfSlice() : " stores offsets : " + startOffset;
         if (storeOffsets) {
           int pos = sliceReader.readInt();

Modified: lucene/dev/lucene2878/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java (original)
+++ lucene/dev/lucene2878/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java Wed Nov 12 14:56:17 2014
@@ -42,7 +42,7 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.CompositeReader;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexOptions;
@@ -198,9 +198,9 @@ public class TestMemoryIndexAgainstRAMDi
           while(iwTermsIter.next() != null) {
             assertNotNull(memTermsIter.next());
             assertEquals(iwTermsIter.term(), memTermsIter.term());
-            DocsAndPositionsEnum iwDocsAndPos = iwTermsIter.docsAndPositions(null, null);
-            DocsAndPositionsEnum memDocsAndPos = memTermsIter.docsAndPositions(null, null);
-            while(iwDocsAndPos.nextDoc() != DocsAndPositionsEnum.NO_MORE_DOCS) {
+            DocsEnum iwDocsAndPos = iwTermsIter.docsAndPositions(null, null);
+            DocsEnum memDocsAndPos = memTermsIter.docsAndPositions(null, null);
+            while(iwDocsAndPos.nextDoc() != DocsEnum.NO_MORE_DOCS) {
               assertEquals(iwDocsAndPos.docID(), memDocsAndPos.nextDoc());
               assertEquals(iwDocsAndPos.freq(), memDocsAndPos.freq());
               for (int i = 0; i < iwDocsAndPos.freq(); i++) {
@@ -219,7 +219,7 @@ public class TestMemoryIndexAgainstRAMDi
             assertEquals(iwTermsIter.term(), memTermsIter.term());
             DocsEnum iwDocsAndPos = iwTermsIter.docs(null, null);
             DocsEnum memDocsAndPos = memTermsIter.docs(null, null);
-            while(iwDocsAndPos.nextDoc() != DocsAndPositionsEnum.NO_MORE_DOCS) {
+            while(iwDocsAndPos.nextDoc() != DocsEnum.NO_MORE_DOCS) {
               assertEquals(iwDocsAndPos.docID(), memDocsAndPos.nextDoc());
               assertEquals(iwDocsAndPos.freq(), memDocsAndPos.freq());
             }
@@ -345,7 +345,7 @@ public class TestMemoryIndexAgainstRAMDi
       memory.addField("foo", "bar", analyzer);
       LeafReader reader = (LeafReader) memory.createSearcher().getIndexReader();
       assertEquals(1, reader.terms("foo").getSumTotalTermFreq());
-      DocsAndPositionsEnum disi = reader.termPositionsEnum(new Term("foo", "bar"));
+      DocsEnum disi = reader.termPositionsEnum(new Term("foo", "bar"));
       int docid = disi.docID();
       assertEquals(-1, docid);
       assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);

Modified: lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java (original)
+++ lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/Sorter.java Wed Nov 12 14:56:17 2014
@@ -17,11 +17,8 @@ package org.apache.lucene.index.sorter;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Comparator;
-
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
@@ -30,6 +27,9 @@ import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
+import java.io.IOException;
+import java.util.Comparator;
+
 /**
  * Sorts documents of a given index by returning a permutation on the document
  * IDs.
@@ -270,6 +270,11 @@ final class Sorter {
     public int freq() throws IOException { throw new UnsupportedOperationException(); }
 
     @Override
+    public int nextPosition() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
     public int docID() { throw new UnsupportedOperationException(); }
 
     @Override

Modified: lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java (original)
+++ lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/index/sorter/SortingLeafReader.java Wed Nov 12 14:56:17 2014
@@ -23,7 +23,6 @@ import java.util.Arrays;
 import org.apache.lucene.index.FilterLeafReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfos;
@@ -49,6 +48,9 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 
+import java.io.IOException;
+import java.util.Arrays;
+
 /**
  * An {@link org.apache.lucene.index.LeafReader} which supports sorting documents by a given
  * {@link Sort}. You can use this class to sort an index as follows:
@@ -164,30 +166,8 @@ public class SortingLeafReader extends F
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, final int flags) throws IOException {
-      final DocsAndPositionsEnum inReuse;
-      final SortingDocsAndPositionsEnum wrapReuse;
-      if (reuse != null && reuse instanceof SortingDocsAndPositionsEnum) {
-        // if we're asked to reuse the given DocsEnum and it is Sorting, return
-        // the wrapped one, since some Codecs expect it.
-        wrapReuse = (SortingDocsAndPositionsEnum) reuse;
-        inReuse = wrapReuse.getWrapped();
-      } else {
-        wrapReuse = null;
-        inReuse = reuse;
-      }
-
-      final DocsAndPositionsEnum inDocsAndPositions = in.docsAndPositions(newToOld(liveDocs), inReuse, flags);
-      if (inDocsAndPositions == null) {
-        return null;
-      }
-
-      // we ignore the fact that offsets may be stored but not asked for,
-      // since this code is expected to be used during addIndexes which will
-      // ask for everything. if that assumption changes in the future, we can
-      // factor in whether 'flags' says offsets are not required.
-      final boolean storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-      return new SortingDocsAndPositionsEnum(docMap.size(), wrapReuse, inDocsAndPositions, docMap, storeOffsets);
+    public DocsEnum docsAndPositions(Bits liveDocs, DocsEnum reuse, final int flags) throws IOException {
+      return docs(liveDocs, reuse, flags);
     }
 
   }
@@ -505,8 +485,9 @@ public class SortingLeafReader extends F
       return in;
     }
   }
-  
-  static class SortingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
+
+  // nocommit - merge this into SortingDocsEnum?
+  static class SortingDocsAndPositionsEnum extends FilterDocsEnum {
     
     /**
      * A {@link TimSorter} which sorts two parallel arrays of doc IDs and
@@ -589,7 +570,7 @@ public class SortingLeafReader extends F
 
     private final RAMFile file;
 
-    SortingDocsAndPositionsEnum(int maxDoc, SortingDocsAndPositionsEnum reuse, final DocsAndPositionsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
+    SortingDocsAndPositionsEnum(int maxDoc, SortingDocsAndPositionsEnum reuse, final DocsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
       super(in);
       this.maxDoc = maxDoc;
       this.storeOffsets = storeOffsets;
@@ -632,14 +613,14 @@ public class SortingLeafReader extends F
     }
 
     // for testing
-    boolean reused(DocsAndPositionsEnum other) {
+    boolean reused(DocsEnum other) {
       if (other == null || !(other instanceof SortingDocsAndPositionsEnum)) {
         return false;
       }
       return docs == ((SortingDocsAndPositionsEnum) other).docs;
     }
 
-    private void addPositions(final DocsAndPositionsEnum in, final IndexOutput out) throws IOException {
+    private void addPositions(final DocsEnum in, final IndexOutput out) throws IOException {
       int freq = in.freq();
       out.writeVInt(freq);
       int previousPosition = 0;
@@ -730,8 +711,8 @@ public class SortingLeafReader extends F
       return startOffset;
     }
 
-    /** Returns the wrapped {@link DocsAndPositionsEnum}. */
-    DocsAndPositionsEnum getWrapped() {
+    /** Returns the wrapped {@link org.apache.lucene.index.DocsEnum}. */
+    DocsEnum getWrapped() {
       return in;
     }
   }

Modified: lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java (original)
+++ lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java Wed Nov 12 14:56:17 2014
@@ -17,16 +17,9 @@
 
 package org.apache.lucene.uninverting;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.lucene.codecs.PostingsFormat; // javadocs
+import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
@@ -41,6 +34,12 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.StringHelper;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
 /**
  * This class enables fast access to multiple term ords for
  * a specified field across all docIDs.
@@ -634,7 +633,7 @@ public class DocTermOrds implements Acco
     }
 
     @Override    
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+    public DocsEnum docsAndPositions(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
       return termsEnum.docsAndPositions(liveDocs, reuse, flags);
     }
 

Modified: lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java (original)
+++ lucene/dev/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java Wed Nov 12 14:56:17 2014
@@ -17,15 +17,6 @@ package org.apache.lucene.uninverting;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.WeakHashMap;
-
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
@@ -51,6 +42,15 @@ import org.apache.lucene.util.packed.Gro
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
 
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.WeakHashMap;
+
 /**
  * Expert: The default cache implementation, storing all values in memory.
  * A WeakHashMap is used for storage.

Modified: lucene/dev/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java (original)
+++ lucene/dev/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/SorterTestBase.java Wed Nov 12 14:56:17 2014
@@ -17,13 +17,6 @@ package org.apache.lucene.index.sorter;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Random;
-
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
@@ -31,8 +24,8 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
@@ -42,7 +35,6 @@ import org.apache.lucene.document.String
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexOptions;
@@ -57,9 +49,8 @@ import org.apache.lucene.index.SortedNum
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.sorter.SortingLeafReader.SortingDocsAndPositionsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.index.sorter.SortingLeafReader.SortingDocsEnum;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -75,6 +66,13 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
 public abstract class SorterTestBase extends LuceneTestCase {
 
   static final class NormsSimilarity extends Similarity {
@@ -254,7 +252,7 @@ public abstract class SorterTestBase ext
   public void testDocsAndPositionsEnum() throws Exception {
     TermsEnum termsEnum = sortedReader.terms(DOC_POSITIONS_FIELD).iterator(null);
     assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
-    DocsAndPositionsEnum sortedPositions = termsEnum.docsAndPositions(null, null);
+    DocsEnum sortedPositions = termsEnum.docsAndPositions(null, null);
     int doc;
     
     // test nextDoc()
@@ -270,10 +268,10 @@ public abstract class SorterTestBase ext
     }
     
     // test advance()
-    final DocsAndPositionsEnum reuse = sortedPositions;
+    final DocsEnum reuse = sortedPositions;
     sortedPositions = termsEnum.docsAndPositions(null, reuse);
-    if (sortedPositions instanceof SortingDocsAndPositionsEnum) {
-      assertTrue(((SortingDocsAndPositionsEnum) sortedPositions).reused(reuse)); // make sure reuse worked
+    if (sortedPositions instanceof SortingDocsEnum) {
+      assertTrue(((SortingDocsEnum) sortedPositions).reused(reuse)); // make sure reuse worked
     }
     doc = 0;
     while ((doc = sortedPositions.advance(doc + TestUtil.nextInt(random(), 1, 5))) != DocIdSetIterator.NO_MORE_DOCS) {

Modified: lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java (original)
+++ lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java Wed Nov 12 14:56:17 2014
@@ -17,26 +17,26 @@ package org.apache.lucene.queries;
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-import java.util.Arrays;
-
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.FunctionQuery;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+
 /**
  * Query that sets document score as a programmatic function of several (sub) scores:
  * <ol>
@@ -234,14 +234,14 @@ public class CustomScoreQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      Scorer subQueryScorer = subQueryWeight.scorer(context, acceptDocs);
+    public Scorer scorer(LeafReaderContext context, int flags, Bits acceptDocs) throws IOException {
+      Scorer subQueryScorer = subQueryWeight.scorer(context, flags, acceptDocs);
       if (subQueryScorer == null) {
         return null;
       }
       Scorer[] valSrcScorers = new Scorer[valSrcWeights.length];
       for(int i = 0; i < valSrcScorers.length; i++) {
-         valSrcScorers[i] = valSrcWeights[i].scorer(context, acceptDocs);
+         valSrcScorers[i] = valSrcWeights[i].scorer(context, flags, acceptDocs);
       }
       return new CustomScorer(CustomScoreQuery.this.getCustomScoreProvider(context), this, queryWeight, subQueryScorer, valSrcScorers);
     }
@@ -333,6 +333,11 @@ public class CustomScoreQuery extends Qu
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return subQueryScorer.nextPosition();
+    }
+
+    @Override
     public Collection<ChildScorer> getChildren() {
       return Collections.singleton(new ChildScorer(subQueryScorer, "CUSTOM"));
     }

Modified: lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java (original)
+++ lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java Wed Nov 12 14:56:17 2014
@@ -17,18 +17,23 @@ package org.apache.lucene.queries.functi
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.*;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.search.ComplexExplanation;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Set;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * Query that is boosted by a ValueSource
@@ -97,8 +102,8 @@ public class BoostedQuery extends Query 
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      Scorer subQueryScorer = qWeight.scorer(context, acceptDocs);
+    public Scorer scorer(LeafReaderContext context, int flags, Bits acceptDocs) throws IOException {
+      Scorer subQueryScorer = qWeight.scorer(context, flags, acceptDocs);
       if (subQueryScorer == null) {
         return null;
       }
@@ -170,6 +175,11 @@ public class BoostedQuery extends Query 
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return scorer.nextPosition();
+    }
+
+    @Override
     public Collection<ChildScorer> getChildren() {
       return Collections.singleton(new ChildScorer(scorer, "CUSTOM"));
     }

Modified: lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java (original)
+++ lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java Wed Nov 12 14:56:17 2014
@@ -17,15 +17,21 @@ package org.apache.lucene.queries.functi
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.ComplexExplanation;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
-import java.util.Set;
 import java.util.Map;
+import java.util.Set;
 
 
 /**
@@ -89,13 +95,13 @@ public class FunctionQuery extends Query
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, int flags, Bits acceptDocs) throws IOException {
       return new AllScorer(context, acceptDocs, this, queryWeight);
     }
 
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      return ((AllScorer)scorer(context, context.reader().getLiveDocs())).explain(doc);
+      return ((AllScorer)scorer(context, DocsEnum.FLAG_FREQS, context.reader().getLiveDocs())).explain(doc);
     }
   }
 
@@ -166,6 +172,11 @@ public class FunctionQuery extends Query
       return 1;
     }
 
+    @Override
+    public int nextPosition() throws IOException {
+      return -1;
+    }
+
     public Explanation explain(int doc) throws IOException {
       float sc = qWeight * vals.floatVal(doc);
 
@@ -177,6 +188,7 @@ public class FunctionQuery extends Query
       result.addDetail(new Explanation(weight.queryNorm,"queryNorm"));
       return result;
     }
+
   }
 
 

Modified: lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java (original)
+++ lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java Wed Nov 12 14:56:17 2014
@@ -93,6 +93,11 @@ public class ValueSourceScorer extends S
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
   public long cost() {
     return maxDoc;
   }

Modified: lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java (original)
+++ lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java Wed Nov 12 14:56:17 2014
@@ -17,12 +17,16 @@
 
 package org.apache.lucene.queries.function.valuesource;
 
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.FloatDocValues;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueFloat;
@@ -123,7 +127,7 @@ class QueryDocValues extends FloatDocVal
     try {
       if (doc < lastDocRequested) {
         if (noMatches) return defVal;
-        scorer = weight.scorer(readerContext, acceptDocs);
+        scorer = weight.scorer(readerContext, DocsEnum.FLAG_FREQS, acceptDocs);
         if (scorer==null) {
           noMatches = true;
           return defVal;
@@ -154,7 +158,7 @@ class QueryDocValues extends FloatDocVal
     try {
       if (doc < lastDocRequested) {
         if (noMatches) return false;
-        scorer = weight.scorer(readerContext, acceptDocs);
+        scorer = weight.scorer(readerContext, DocsEnum.FLAG_FREQS, acceptDocs);
         scorerDoc = -1;
         if (scorer==null) {
           noMatches = true;
@@ -212,7 +216,7 @@ class QueryDocValues extends FloatDocVal
             mval.exists = false;
             return;
           }
-          scorer = weight.scorer(readerContext, acceptDocs);
+          scorer = weight.scorer(readerContext, DocsEnum.FLAG_FREQS, acceptDocs);
           scorerDoc = -1;
           if (scorer==null) {
             noMatches = true;

Modified: lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java (original)
+++ lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java Wed Nov 12 14:56:17 2014
@@ -84,6 +84,11 @@ public class TFValueSource extends TermF
             }
 
             @Override
+            public int nextPosition() throws IOException {
+              return -1;
+            }
+
+            @Override
             public int docID() {
               return DocIdSetIterator.NO_MORE_DOCS;
             }

Modified: lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java (original)
+++ lucene/dev/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java Wed Nov 12 14:56:17 2014
@@ -77,6 +77,11 @@ public class TermFreqValueSource extends
             }
 
             @Override
+            public int nextPosition() throws IOException {
+              return -1;
+            }
+
+            @Override
             public int docID() {
               return DocIdSetIterator.NO_MORE_DOCS;
             }

Modified: lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java (original)
+++ lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java Wed Nov 12 14:56:17 2014
@@ -17,13 +17,9 @@ package org.apache.lucene.codecs.idversi
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.util.Collections;
-
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.PostingsReaderBase;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentReadState;
@@ -32,6 +28,9 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Bits;
 
+import java.io.IOException;
+import java.util.Collections;
+
 final class IDVersionPostingsReader extends PostingsReaderBase {
 
   @Override
@@ -80,8 +79,8 @@ final class IDVersionPostingsReader exte
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, int flags) {
+  public DocsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs,
+                                   DocsEnum reuse, int flags) {
     SingleDocsAndPositionsEnum posEnum;
 
     if (reuse instanceof SingleDocsAndPositionsEnum) {

Modified: lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java (original)
+++ lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java Wed Nov 12 14:56:17 2014
@@ -17,11 +17,7 @@ package org.apache.lucene.codecs.idversi
  * limitations under the License.
  */
 
-import java.io.IOException;
-import java.io.PrintStream;
-
 import org.apache.lucene.codecs.BlockTermState;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.TermState;
@@ -37,6 +33,9 @@ import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
 import org.apache.lucene.util.fst.Util;
 
+import java.io.IOException;
+import java.io.PrintStream;
+
 /** Iterates through terms in this field; this class is public so users
  *  can cast it to call {@link #seekExact(BytesRef, long)} for
  *  optimistic-concurreny, and also {@link #getVersion} to get the
@@ -1010,7 +1009,7 @@ public final class IDVersionSegmentTerms
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+  public DocsEnum docsAndPositions(Bits skipDocs, DocsEnum reuse, int flags) throws IOException {
     if (fr.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
       // Positions were not indexed:
       return null;

Modified: lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsAndPositionsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsAndPositionsEnum.java?rev=1638805&r1=1638804&r2=1638805&view=diff
==============================================================================
--- lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsAndPositionsEnum.java (original)
+++ lucene/dev/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsAndPositionsEnum.java Wed Nov 12 14:56:17 2014
@@ -17,11 +17,11 @@ package org.apache.lucene.codecs.idversi
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
-class SingleDocsAndPositionsEnum extends DocsAndPositionsEnum {
+class SingleDocsAndPositionsEnum extends DocsEnum {
   private int doc;
   private int pos;
   private int singleDocID;