You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/13 06:06:27 UTC

svn commit: r1513336 [9/11] - in /lucene/dev/branches/lucene3069/lucene: ./ analysis/ analysis/common/ analysis/common/src/java/org/apache/lucene/analysis/charfilter/ analysis/common/src/java/org/apache/lucene/analysis/hunspell/ analysis/common/src/jav...

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldQuery.java Tue Aug 13 04:06:18 2013
@@ -17,6 +17,8 @@ package org.apache.lucene.search.vectorh
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -39,6 +41,7 @@ import org.apache.lucene.search.PhraseQu
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.vectorhighlight.FieldTermStack.TermInfo;
+import org.apache.lucene.util.InPlaceMergeSorter;
 
 /**
  * FieldQuery breaks down query object into terms/phrases and keeps
@@ -330,7 +333,8 @@ public class FieldQuery {
     return root.searchPhrase( phraseCandidate );
   }
   
-  private QueryPhraseMap getRootMap( String fieldName ){
+  /** Get the root map for the given field name. */
+  public QueryPhraseMap getRootMap( String fieldName ){
     return rootMaps.get( fieldMatch ? fieldName : null );
   }
   
@@ -347,6 +351,7 @@ public class FieldQuery {
     boolean terminal;
     int slop;   // valid if terminal == true and phraseHighlight == true
     float boost;  // valid if terminal == true
+    int[] positions; // valid if terminal == true
     int termOrPhraseNumber;   // valid if terminal == true
     FieldQuery fieldQuery;
     Map<String, QueryPhraseMap> subMap = new HashMap<String, QueryPhraseMap>();
@@ -369,38 +374,107 @@ public class FieldQuery {
       return map;
     }
 
-      void add( Query query, IndexReader reader ) {
+    void add( Query query, IndexReader reader ) {
       if( query instanceof TermQuery ){
         addTerm( ((TermQuery)query).getTerm(), query.getBoost() );
       }
       else if( query instanceof PhraseQuery ){
         PhraseQuery pq = (PhraseQuery)query;
-        Term[] terms = pq.getTerms();
-        Map<String, QueryPhraseMap> map = subMap;
-        QueryPhraseMap qpm = null;
-        for( Term term : terms ){
-          qpm = getOrNewMap( map, term.text() );
-          map = qpm.subMap;
-        }
-        qpm.markTerminal( pq.getSlop(), pq.getBoost() );
+        final Term[] terms = pq.getTerms();
+        final int[] positions = pq.getPositions();
+        new InPlaceMergeSorter() {
+
+          @Override
+          protected void swap(int i, int j) {
+            Term tmpTerm = terms[i];
+            terms[i] = terms[j];
+            terms[j] = tmpTerm;
+
+            int tmpPos = positions[i];
+            positions[i] = positions[j];
+            positions[j] = tmpPos;
+          }
+
+          @Override
+          protected int compare(int i, int j) {
+            return positions[i] - positions[j];
+          }
+        }.sort(0, terms.length);
+
+        addToMap(pq, terms, positions, 0, subMap, pq.getSlop());
       }
       else
         throw new RuntimeException( "query \"" + query.toString() + "\" must be flatten first." );
     }
-    
+
+    private int numTermsAtSamePosition(int[] positions, int i) {
+      int numTermsAtSamePosition = 1;
+      for (int j = i + 1; j < positions.length; ++j) {
+        if (positions[j] == positions[i]) {
+          ++numTermsAtSamePosition;
+        }
+      }
+      return numTermsAtSamePosition;
+    }
+
+    private void addToMap(PhraseQuery pq, Term[] terms, int[] positions, int i, Map<String, QueryPhraseMap> map, int slop) {
+      int numTermsAtSamePosition = numTermsAtSamePosition(positions, i);
+      for (int j = 0; j < numTermsAtSamePosition; ++j) {
+        QueryPhraseMap qpm = getOrNewMap(map, terms[i + j].text());
+        if (i + numTermsAtSamePosition == terms.length) {
+          qpm.markTerminal(pq.getSlop(), pq.getBoost(), uniquePositions(positions));
+        } else {
+          addToMap(pq, terms, positions, i + numTermsAtSamePosition, qpm.subMap, slop);
+        }
+      }
+      if (slop > 2 && i + numTermsAtSamePosition < terms.length) {
+        Term[] otherTerms = Arrays.copyOf(terms, terms.length);
+        int[] otherPositions = Arrays.copyOf(positions, positions.length);
+        final int nextTermAtSamePosition = numTermsAtSamePosition(positions, i + numTermsAtSamePosition);
+        System.arraycopy(terms, i + numTermsAtSamePosition, otherTerms, i, nextTermAtSamePosition);
+        System.arraycopy(positions, i + numTermsAtSamePosition, otherPositions, i, nextTermAtSamePosition);
+        System.arraycopy(terms, i, otherTerms, i + nextTermAtSamePosition, numTermsAtSamePosition);
+        System.arraycopy(positions, i, otherPositions, i + nextTermAtSamePosition, numTermsAtSamePosition);
+        addToMap(pq, otherTerms, otherPositions, i, map, slop - 2);
+      }
+    }
+
+    private int[] uniquePositions(int[] positions) {
+      int uniqueCount = 1;
+      for (int i = 1; i < positions.length; ++i) {
+        if (positions[i] != positions[i - 1]) {
+          ++uniqueCount;
+        }
+      }
+      if (uniqueCount == positions.length) {
+        return positions;
+      }
+      int[] result = new int[uniqueCount];
+      result[0] = positions[0];
+      for (int i = 1, j = 1; i < positions.length; ++i) {
+        if (positions[i] != positions[i - 1]) {
+          result[j++] = positions[i];
+        }
+      }
+      return result;
+    }
+
     public QueryPhraseMap getTermMap( String term ){
       return subMap.get( term );
     }
     
     private void markTerminal( float boost ){
-      markTerminal( 0, boost );
+      markTerminal( 0, boost, null );
     }
     
-    private void markTerminal( int slop, float boost ){
-      this.terminal = true;
-      this.slop = slop;
-      this.boost = boost;
-      this.termOrPhraseNumber = fieldQuery.nextTermOrPhraseNumber();
+    private void markTerminal( int slop, float boost, int[] positions ){
+      if (slop > this.slop || (slop == this.slop && boost > this.boost)) {
+        this.terminal = true;
+        this.slop = slop;
+        this.boost = boost;
+        this.termOrPhraseNumber = fieldQuery.nextTermOrPhraseNumber();
+        this.positions = positions;
+      }
     }
     
     public boolean isTerminal(){
@@ -435,15 +509,20 @@ public class FieldQuery {
       // if the candidate is a term, it is valid
       if( phraseCandidate.size() == 1 ) return true;
 
+      
+      assert phraseCandidate.size() == positions.length;
       // else check whether the candidate is valid phrase
       // compare position-gaps between terms to slop
       int pos = phraseCandidate.get( 0 ).getPosition();
+      int totalDistance = 0;
       for( int i = 1; i < phraseCandidate.size(); i++ ){
         int nextPos = phraseCandidate.get( i ).getPosition();
-        if( Math.abs( nextPos - pos - 1 ) > slop ) return false;
+        final int expectedDelta = positions[i] - positions[i - 1];
+        final int actualDelta = nextPos - pos;
+        totalDistance += Math.abs(expectedDelta - actualDelta);
         pos = nextPos;
       }
-      return true;
+      return totalDistance <= slop;
     }
   }
 }

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/java/org/apache/lucene/search/vectorhighlight/FieldTermStack.java Tue Aug 13 04:06:18 2013
@@ -145,6 +145,13 @@ public class FieldTermStack {
   }
 
   /**
+   * Return the top TermInfo object of the stack without removing it.
+   */
+  public TermInfo peek() {
+    return termList.peek();
+  }
+
+  /**
    * @param termInfo the TermInfo object to be put on the top of the stack
    */
   public void push( TermInfo termInfo ){

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighter.java Tue Aug 13 04:06:18 2013
@@ -87,6 +87,81 @@ public class TestPostingsHighlighter ext
     dir.close();
   }
   
+  public void testFormatWithMatchExceedingContentLength() throws Exception {
+          
+    int maxLength = 17;
+    String bodyText = "123 5678 01234 TEST";
+    
+    final Analyzer analyzer = new MockAnalyzer(random());
+    
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    final FieldType fieldType = new FieldType(TextField.TYPE_STORED);
+    fieldType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    final Field body = new Field("body", bodyText, fieldType);
+    
+    Document doc = new Document();
+    doc.add(body);
+    
+    iw.addDocument(doc);
+    
+    IndexReader ir = iw.getReader();
+    iw.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    
+    Query query = new TermQuery(new Term("body", "test"));
+    
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    
+    PostingsHighlighter highlighter = new PostingsHighlighter(maxLength);
+    String snippets[] = highlighter.highlight("body", query, searcher, topDocs);
+    
+    
+    assertEquals(1, snippets.length);
+    // LUCENE-5166: no snippet
+    assertEquals("123 5678 01234 TE", snippets[0]);
+    
+    ir.close();
+    dir.close();
+  }
+  
+  // simple test highlighting last word.
+  public void testHighlightLastWord() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setMergePolicy(newLogMergePolicy());
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc);
+    
+    FieldType offsetsType = new FieldType(TextField.TYPE_STORED);
+    offsetsType.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    Field body = new Field("body", "", offsetsType);
+    Document doc = new Document();
+    doc.add(body);
+    
+    body.setStringValue("This is a test");
+    iw.addDocument(doc);
+    
+    IndexReader ir = iw.getReader();
+    iw.close();
+    
+    IndexSearcher searcher = newSearcher(ir);
+    PostingsHighlighter highlighter = new PostingsHighlighter();
+    Query query = new TermQuery(new Term("body", "test"));
+    TopDocs topDocs = searcher.search(query, null, 10, Sort.INDEXORDER);
+    assertEquals(1, topDocs.totalHits);
+    String snippets[] = highlighter.highlight("body", query, searcher, topDocs);
+    assertEquals(1, snippets.length);
+    assertEquals("This is a <b>test</b>", snippets[0]);
+    
+    ir.close();
+    dir.close();
+  }
+  
   // simple test with one sentence documents.
   public void testOneSentence() throws Exception {
     Directory dir = newDirectory();

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighterRanking.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighterRanking.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighterRanking.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/postingshighlight/TestPostingsHighlighterRanking.java Tue Aug 13 04:06:18 2013
@@ -173,6 +173,8 @@ public class TestPostingsHighlighterRank
         assertTrue(p.getNumMatches() > 0);
         assertTrue(p.getStartOffset() >= 0);
         assertTrue(p.getStartOffset() <= content.length());
+        assertTrue(p.getEndOffset() >= p.getStartOffset());
+        assertTrue(p.getEndOffset() <= content.length());
         // we use a very simple analyzer. so we can assert the matches are correct
         int lastMatchStart = -1;
         for (int i = 0; i < p.getNumMatches(); i++) {

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FastVectorHighlighterTest.java Tue Aug 13 04:06:18 2013
@@ -16,10 +16,18 @@ package org.apache.lucene.search.vectorh
  * limitations under the License.
  */
 import java.io.IOException;
+import java.io.Reader;
+import java.util.Arrays;
 
+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.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
@@ -27,12 +35,14 @@ import org.apache.lucene.document.TextFi
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.CommonTermsQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
@@ -40,7 +50,8 @@ import org.apache.lucene.util.LuceneTest
 
 
 public class FastVectorHighlighterTest extends LuceneTestCase {
-  
+
+  private static final String FIELD = "text";
   
   public void testSimpleHighlightTest() throws IOException {
     Directory dir = newDirectory();
@@ -287,4 +298,128 @@ public class FastVectorHighlighterTest e
     writer.close();
     dir.close();
   }
+
+  public void testOverlappingPhrases() throws IOException {
+    final Analyzer analyzer = new Analyzer() {
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+        final Tokenizer source = new MockTokenizer(reader);
+        TokenStream sink = source;
+        sink = new SynonymFilter(sink);
+        return new TokenStreamComponents(source, sink);
+      }
+
+    };
+    final Directory directory = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, analyzer);
+    Document doc = new Document();
+    FieldType withVectors = new FieldType(TextField.TYPE_STORED);
+    withVectors.setStoreTermVectors(true);
+    withVectors.setStoreTermVectorPositions(true);
+    withVectors.setStoreTermVectorOffsets(true);
+    doc.add(new Field(FIELD, "a b c", withVectors));
+    iw.addDocument(doc);
+    DirectoryReader ir = iw.getReader();
+
+    // Disjunction of two overlapping phrase queries
+    final PhraseQuery pq1 = new PhraseQuery();
+    pq1.add(new Term(FIELD, "a"), 0);
+    pq1.add(new Term(FIELD, "b"), 1);
+    pq1.add(new Term(FIELD, "c"), 2);
+
+    final PhraseQuery pq2 = new PhraseQuery();
+    pq2.add(new Term(FIELD, "a"), 0);
+    pq2.add(new Term(FIELD, "B"), 1);
+    pq2.add(new Term(FIELD, "c"), 2);
+
+    final BooleanQuery bq = new BooleanQuery();
+    bq.add(pq1, Occur.SHOULD);
+    bq.add(pq2, Occur.SHOULD);
+
+    // Single phrase query with two terms at the same position
+    final PhraseQuery pq = new PhraseQuery();
+    pq.add(new Term(FIELD, "a"), 0);
+    pq.add(new Term(FIELD, "b"), 1);
+    pq.add(new Term(FIELD, "B"), 1);
+    pq.add(new Term(FIELD, "c"), 2);
+
+    for (Query query : Arrays.asList(pq1, pq2, bq, pq)) {
+      assertEquals(1, new IndexSearcher(ir).search(bq, 1).totalHits);
+
+      FastVectorHighlighter highlighter = new FastVectorHighlighter();
+      FieldQuery fieldQuery  = highlighter.getFieldQuery(query, ir);
+      String[] bestFragments = highlighter.getBestFragments(fieldQuery, ir, 0, FIELD, 1000, 1);
+      assertEquals("<b>a b c</b>", bestFragments[0]);
+    }
+
+    ir.close();
+    iw.close();
+    directory.close();
+  }
+
+  public void testPhraseWithGap() throws IOException {
+    final Directory directory = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), directory, new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false));
+    Document doc = new Document();
+    FieldType withVectors = new FieldType(TextField.TYPE_STORED);
+    withVectors.setStoreTermVectors(true);
+    withVectors.setStoreTermVectorPositions(true);
+    withVectors.setStoreTermVectorOffsets(true);
+    doc.add(new Field(FIELD, "a b c", withVectors));
+    iw.addDocument(doc);
+    DirectoryReader ir = iw.getReader();
+
+    final PhraseQuery pq = new PhraseQuery();
+    pq.add(new Term(FIELD, "c"), 2);
+    pq.add(new Term(FIELD, "a"), 0);
+
+    assertEquals(1, new IndexSearcher(ir).search(pq, 1).totalHits);
+
+    FastVectorHighlighter highlighter = new FastVectorHighlighter();
+    FieldQuery fieldQuery  = highlighter.getFieldQuery(pq, ir);
+    String[] bestFragments = highlighter.getBestFragments(fieldQuery, ir, 0, FIELD, 1000, 1);
+    assertEquals("<b>a</b> b <b>c</b>", bestFragments[0]);
+
+    ir.close();
+    iw.close();
+    directory.close();
+  }
+
+  // Simple token filter that adds 'B' as a synonym of 'b'
+  private static class SynonymFilter extends TokenFilter {
+
+    final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+    final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+
+    State pending;
+
+    protected SynonymFilter(TokenStream input) {
+      super(input);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      if (pending != null) {
+        restoreState(pending);
+        termAtt.setEmpty().append('B');
+        posIncAtt.setPositionIncrement(0);
+        pending = null;
+        return true;
+      }
+      if (!input.incrementToken()) {
+        return false;
+      }
+      if (termAtt.toString().equals("b")) {
+        pending = captureState();
+      }
+      return true;
+    }
+
+    @Override
+    public void reset() throws IOException {
+      super.reset();
+      pending = null;
+    }
+  }
 }

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldPhraseListTest.java Tue Aug 13 04:06:18 2013
@@ -120,7 +120,31 @@ public class FieldPhraseListTest extends
     assertEquals( 4, fpl.phraseList.get( 0 ).getStartOffset() );
     assertEquals( 9, fpl.phraseList.get( 0 ).getEndOffset() );
   }
-  
+
+  public void testProximityPhraseReverse() throws Exception {
+    make1d1fIndex( "z a a b c" );
+    
+    FieldQuery fq = new FieldQuery( pqF( 2F, 3, "c", "a" ), true, true );
+    FieldTermStack stack = new FieldTermStack( reader, 0, F, fq );
+    FieldPhraseList fpl = new FieldPhraseList( stack, fq );
+    assertEquals( 1, fpl.phraseList.size() );
+    assertEquals( "ac(2.0)((4,5)(8,9))", fpl.phraseList.get( 0 ).toString() );
+    assertEquals( 4, fpl.phraseList.get( 0 ).getStartOffset() );
+    assertEquals( 9, fpl.phraseList.get( 0 ).getEndOffset() );
+  }
+
+  public void testProximityPhraseWithRepeatedTerms() throws Exception {
+    make1d1fIndex( "z a a b b z d" );
+    
+    FieldQuery fq = new FieldQuery( pqF( 2F, 2, "a", "b", "d" ), true, true );
+    FieldTermStack stack = new FieldTermStack( reader, 0, F, fq );
+    FieldPhraseList fpl = new FieldPhraseList( stack, fq );
+    assertEquals( 1, fpl.phraseList.size() );
+    assertEquals( "abd(2.0)((4,7)(12,13))", fpl.phraseList.get( 0 ).toString() );
+    assertEquals( 4, fpl.phraseList.get( 0 ).getStartOffset() );
+    assertEquals( 13, fpl.phraseList.get( 0 ).getEndOffset() );
+  }
+
   public void test2PhrasesOverlap() throws Exception {
     make1d1fIndex( "d a b c d" );
 

Modified: lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/highlighter/src/test/org/apache/lucene/search/vectorhighlight/FieldQueryTest.java Tue Aug 13 04:06:18 2013
@@ -863,8 +863,8 @@ public class FieldQueryTest extends Abst
     phraseCandidate.add( new TermInfo( "c", 4, 5, 4, 1 ) );
     assertNull( fq.searchPhrase( F, phraseCandidate ) );
 
-    // "a b c"~1
-    query = pqF( 1F, 1, "a", "b", "c" );
+    // "a b c"~2
+    query = pqF( 1F, 2, "a", "b", "c" );
 
     // phraseHighlight = true, fieldMatch = true
     fq = new FieldQuery( query, true, true );

Modified: lucene/dev/branches/lucene3069/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java Tue Aug 13 04:06:18 2013
@@ -241,7 +241,7 @@ class TermsIncludingScoreQuery extends Q
         }
 
         scoreUpto = upto;
-        if (termsEnum.seekExact(terms.get(ords[upto++], spare), true)) {
+        if (termsEnum.seekExact(terms.get(ords[upto++], spare))) {
           docsEnum = reuse = termsEnum.docs(acceptDocs, reuse, DocsEnum.FLAG_NONE);
         }
       }
@@ -337,7 +337,7 @@ class TermsIncludingScoreQuery extends Q
       BytesRef spare = new BytesRef();
       DocsEnum docsEnum = null;
       for (int i = 0; i < terms.size(); i++) {
-        if (termsEnum.seekExact(terms.get(ords[i], spare), true)) {
+        if (termsEnum.seekExact(terms.get(ords[i], spare))) {
           docsEnum = termsEnum.docs(acceptDocs, docsEnum, DocsEnum.FLAG_NONE);
           float score = TermsIncludingScoreQuery.this.scores[ords[i]];
           for (int doc = docsEnum.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = docsEnum.nextDoc()) {
@@ -393,7 +393,7 @@ class TermsIncludingScoreQuery extends Q
       BytesRef spare = new BytesRef();
       DocsEnum docsEnum = null;
       for (int i = 0; i < terms.size(); i++) {
-        if (termsEnum.seekExact(terms.get(ords[i], spare), true)) {
+        if (termsEnum.seekExact(terms.get(ords[i], spare))) {
           docsEnum = termsEnum.docs(acceptDocs, docsEnum, DocsEnum.FLAG_NONE);
           float score = TermsIncludingScoreQuery.this.scores[ords[i]];
           for (int doc = docsEnum.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = docsEnum.nextDoc()) {

Modified: lucene/dev/branches/lucene3069/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (original)
+++ lucene/dev/branches/lucene3069/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java Tue Aug 13 04:06:18 2013
@@ -557,7 +557,7 @@ public class TestJoinUtil extends Lucene
             joinValues.addAll(joinValueToJoinScores.keySet());
             for (BytesRef joinValue : joinValues) {
               termsEnum = terms.iterator(termsEnum);
-              if (termsEnum.seekExact(joinValue, true)) {
+              if (termsEnum.seekExact(joinValue)) {
                 docsEnum = termsEnum.docs(slowCompositeReader.getLiveDocs(), docsEnum, DocsEnum.FLAG_NONE);
                 JoinScore joinScore = joinValueToJoinScores.get(joinValue);
 

Modified: lucene/dev/branches/lucene3069/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/branches/lucene3069/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Tue Aug 13 04:06:18 2013
@@ -885,13 +885,13 @@ public class MemoryIndex {
     
 
       @Override
-      public boolean seekExact(BytesRef text, boolean useCache) {
+      public boolean seekExact(BytesRef text) {
         termUpto = binarySearch(text, br, 0, info.terms.size()-1, info.terms, info.sortedTerms, BytesRef.getUTF8SortedAsUnicodeComparator());
         return termUpto >= 0;
       }
 
       @Override
-      public SeekStatus seekCeil(BytesRef text, boolean useCache) {
+      public SeekStatus seekCeil(BytesRef text) {
         termUpto = binarySearch(text, br, 0, info.terms.size()-1, info.terms, info.sortedTerms, BytesRef.getUTF8SortedAsUnicodeComparator());
         if (termUpto < 0) { // not found; choose successor
           termUpto = -termUpto-1;

Modified: lucene/dev/branches/lucene3069/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/memory/src/test/org/apache/lucene/index/memory/MemoryIndexTest.java Tue Aug 13 04:06:18 2013
@@ -327,7 +327,7 @@ public class MemoryIndexTest extends Bas
     
     // now reuse and check again
     TermsEnum te = reader.terms("foo").iterator(null);
-    assertTrue(te.seekExact(new BytesRef("bar"), true));
+    assertTrue(te.seekExact(new BytesRef("bar")));
     disi = te.docs(null, disi, DocsEnum.FLAG_NONE);
     docid = disi.docID();
     assertEquals(-1, docid);
@@ -361,7 +361,7 @@ public class MemoryIndexTest extends Bas
       
       // now reuse and check again
       TermsEnum te = reader.terms("foo").iterator(null);
-      assertTrue(te.seekExact(new BytesRef("bar"), true));
+      assertTrue(te.seekExact(new BytesRef("bar")));
       disi = te.docsAndPositions(null, disi);
       docid = disi.docID();
       assertEquals(-1, docid);

Modified: lucene/dev/branches/lucene3069/lucene/module-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/module-build.xml?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/module-build.xml (original)
+++ lucene/dev/branches/lucene3069/lucene/module-build.xml Tue Aug 13 04:06:18 2013
@@ -132,6 +132,17 @@
     </ant>
     <property name="queryparser-javadocs.uptodate" value="true"/>
   </target>
+	
+  <property name="join.jar" value="${common.dir}/build/join/lucene-join-${version}.jar"/>
+  <target name="check-join-uptodate" unless="join.uptodate">
+    <module-uptodate name="join" jarfile="${join.jar}" property="join.uptodate"/>
+  </target>
+  <target name="jar-join" unless="join.uptodate" depends="check-join-uptodate">
+    <ant dir="${common.dir}/join" target="jar-core" inheritAll="false">
+      <propertyset refid="uptodate.and.compiled.properties"/>
+	</ant>
+	<property name="join.uptodate" value="true"/>
+  </target>	
   
   <property name="analyzers-common.jar" value="${common.dir}/build/analysis/common/lucene-analyzers-common-${version}.jar"/>
   <target name="check-analyzers-common-uptodate" unless="analyzers-common.uptodate">

Modified: lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java Tue Aug 13 04:06:18 2013
@@ -74,7 +74,8 @@ public class CommonTermsQuery extends Qu
   protected final Occur highFreqOccur;
   protected float lowFreqBoost = 1.0f;
   protected float highFreqBoost = 1.0f;
-  protected float minNrShouldMatch = 0;
+  protected float lowFreqMinNrShouldMatch = 0;
+  protected float highFreqMinNrShouldMatch = 0;
   
   /**
    * Creates a new {@link CommonTermsQuery}
@@ -161,10 +162,18 @@ public class CommonTermsQuery extends Qu
   }
   
   protected int calcLowFreqMinimumNumberShouldMatch(int numOptional) {
-      if (minNrShouldMatch >= 1.0f || minNrShouldMatch == 0.0f) {
-          return (int) minNrShouldMatch;
-      }
-      return (int) (Math.round(minNrShouldMatch * numOptional));
+    return minNrShouldMatch(lowFreqMinNrShouldMatch, numOptional);
+  }
+  
+  protected int calcHighFreqMinimumNumberShouldMatch(int numOptional) {
+    return minNrShouldMatch(highFreqMinNrShouldMatch, numOptional);
+  }
+  
+  private final int minNrShouldMatch(float minNrShouldMatch, int numOptional) {
+    if (minNrShouldMatch >= 1.0f || minNrShouldMatch == 0.0f) {
+      return (int) minNrShouldMatch;
+    }
+    return (int) (Math.round(minNrShouldMatch * numOptional));
   }
   
   protected Query buildQuery(final int maxDoc,
@@ -190,11 +199,16 @@ public class CommonTermsQuery extends Qu
       }
       
     }
-    final int numLowFreqClauses = lowFreq.clauses().size(); 
+    final int numLowFreqClauses = lowFreq.clauses().size();
+    final int numHighFreqClauses = highFreq.clauses().size();
     if (lowFreqOccur == Occur.SHOULD && numLowFreqClauses > 0) {
       int minMustMatch = calcLowFreqMinimumNumberShouldMatch(numLowFreqClauses);
       lowFreq.setMinimumNumberShouldMatch(minMustMatch);
     }
+    if (highFreqOccur == Occur.SHOULD && numHighFreqClauses > 0) {
+      int minMustMatch = calcHighFreqMinimumNumberShouldMatch(numHighFreqClauses);
+      highFreq.setMinimumNumberShouldMatch(minMustMatch);
+    }
     if (lowFreq.clauses().isEmpty()) {
       /*
        * if lowFreq is empty we rewrite the high freq terms in a conjunction to
@@ -246,7 +260,7 @@ public class CommonTermsQuery extends Qu
         assert termsEnum != null;
         
         if (termsEnum == TermsEnum.EMPTY) continue;
-        if (termsEnum.seekExact(term.bytes(), false)) {
+        if (termsEnum.seekExact(term.bytes())) {
           if (termContext == null) {
             contextArray[i] = new TermContext(reader.getContext(),
                 termsEnum.termState(), context.ord, termsEnum.docFreq(),
@@ -272,7 +286,7 @@ public class CommonTermsQuery extends Qu
   }
   
   /**
-   * Specifies a minimum number of the optional BooleanClauses which must be
+   * Specifies a minimum number of the low frequent optional BooleanClauses which must be
    * satisfied in order to produce a match on the low frequency terms query
    * part. This method accepts a float value in the range [0..1) as a fraction
    * of the actual query terms in the low frequent clause or a number
@@ -287,16 +301,44 @@ public class CommonTermsQuery extends Qu
    * @param min
    *          the number of optional clauses that must match
    */
-  public void setMinimumNumberShouldMatch(float min) {
-    this.minNrShouldMatch = min;
+  public void setLowFreqMinimumNumberShouldMatch(float min) {
+    this.lowFreqMinNrShouldMatch = min;
   }
   
   /**
-   * Gets the minimum number of the optional BooleanClauses which must be
+   * Gets the minimum number of the optional low frequent BooleanClauses which must be
    * satisfied.
    */
-  public float getMinimumNumberShouldMatch() {
-    return minNrShouldMatch;
+  public float getLowFreqMinimumNumberShouldMatch() {
+    return lowFreqMinNrShouldMatch;
+  }
+  
+  /**
+   * Specifies a minimum number of the high frequent optional BooleanClauses which must be
+   * satisfied in order to produce a match on the low frequency terms query
+   * part. This method accepts a float value in the range [0..1) as a fraction
+   * of the actual query terms in the low frequent clause or a number
+   * <tt>&gt;=1</tt> as an absolut number of clauses that need to match.
+   * 
+   * <p>
+   * By default no optional clauses are necessary for a match (unless there are
+   * no required clauses). If this method is used, then the specified number of
+   * clauses is required.
+   * </p>
+   * 
+   * @param min
+   *          the number of optional clauses that must match
+   */
+  public void setHighFreqMinimumNumberShouldMatch(float min) {
+    this.highFreqMinNrShouldMatch = min;
+  }
+  
+  /**
+   * Gets the minimum number of the optional high frequent BooleanClauses which must be
+   * satisfied.
+   */
+  public float getHighFreqMinimumNumberShouldMatch() {
+    return highFreqMinNrShouldMatch;
   }
   
   @Override
@@ -308,7 +350,7 @@ public class CommonTermsQuery extends Qu
   public String toString(String field) {
     StringBuilder buffer = new StringBuilder();
     boolean needParens = (getBoost() != 1.0)
-        || (getMinimumNumberShouldMatch() > 0);
+        || (getLowFreqMinimumNumberShouldMatch() > 0);
     if (needParens) {
       buffer.append("(");
     }
@@ -321,9 +363,12 @@ public class CommonTermsQuery extends Qu
     if (needParens) {
       buffer.append(")");
     }
-    if (getMinimumNumberShouldMatch() > 0) {
+    if (getLowFreqMinimumNumberShouldMatch() > 0 || getHighFreqMinimumNumberShouldMatch() > 0) {
       buffer.append('~');
-      buffer.append(getMinimumNumberShouldMatch());
+      buffer.append("(");
+      buffer.append(getLowFreqMinimumNumberShouldMatch());
+      buffer.append(getHighFreqMinimumNumberShouldMatch());
+      buffer.append(")");
     }
     if (getBoost() != 1.0f) {
       buffer.append(ToStringUtils.boost(getBoost()));
@@ -343,7 +388,8 @@ public class CommonTermsQuery extends Qu
     result = prime * result
         + ((lowFreqOccur == null) ? 0 : lowFreqOccur.hashCode());
     result = prime * result + Float.floatToIntBits(maxTermFrequency);
-    result = prime * result + Float.floatToIntBits(minNrShouldMatch);
+    result = prime * result + Float.floatToIntBits(lowFreqMinNrShouldMatch);
+    result = prime * result + Float.floatToIntBits(highFreqMinNrShouldMatch);
     result = prime * result + ((terms == null) ? 0 : terms.hashCode());
     return result;
   }
@@ -363,7 +409,8 @@ public class CommonTermsQuery extends Qu
     if (lowFreqOccur != other.lowFreqOccur) return false;
     if (Float.floatToIntBits(maxTermFrequency) != Float
         .floatToIntBits(other.maxTermFrequency)) return false;
-    if (minNrShouldMatch != other.minNrShouldMatch) return false;
+    if (lowFreqMinNrShouldMatch != other.lowFreqMinNrShouldMatch) return false;
+    if (highFreqMinNrShouldMatch != other.highFreqMinNrShouldMatch) return false;
     if (terms == null) {
       if (other.terms != null) return false;
     } else if (!terms.equals(other.terms)) return false;

Modified: lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java Tue Aug 13 04:06:18 2013
@@ -193,7 +193,7 @@ public final class TermsFilter extends F
         for (int i = termsAndField.start; i < termsAndField.end; i++) {
           spare.offset = offsets[i];
           spare.length = offsets[i+1] - offsets[i];
-          if (termsEnum.seekExact(spare, false)) { // don't use cache since we could pollute the cache here easily
+          if (termsEnum.seekExact(spare)) {
             docs = termsEnum.docs(acceptDocs, docs, DocsEnum.FLAG_NONE); // no freq since we don't need them
             if (result == null) {
               if (docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {

Modified: lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/JoinDocFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/JoinDocFreqValueSource.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/JoinDocFreqValueSource.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/JoinDocFreqValueSource.java Tue Aug 13 04:06:18 2013
@@ -69,7 +69,7 @@ public class JoinDocFreqValueSource exte
       {
         try {
           terms.get(doc, ref);
-          if (termsEnum.seekExact(ref, true)) {
+          if (termsEnum.seekExact(ref)) {
             return termsEnum.docFreq();
           } else {
             return 0;

Modified: lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java Tue Aug 13 04:06:18 2013
@@ -67,7 +67,7 @@ public class TFValueSource extends TermF
         
         if (terms != null) {
           final TermsEnum termsEnum = terms.iterator(null);
-          if (termsEnum.seekExact(indexedBytes, false)) {
+          if (termsEnum.seekExact(indexedBytes)) {
             docs = termsEnum.docs(null, null);
           } else {
             docs = null;

Modified: lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java Tue Aug 13 04:06:18 2013
@@ -60,7 +60,7 @@ public class TermFreqValueSource extends
         
         if (terms != null) {
           final TermsEnum termsEnum = terms.iterator(null);
-          if (termsEnum.seekExact(indexedBytes, false)) {
+          if (termsEnum.seekExact(indexedBytes)) {
             docs = termsEnum.docs(null, null);
           } else {
             docs = null;

Modified: lucene/dev/branches/lucene3069/lucene/queries/src/test/org/apache/lucene/queries/CommonTermsQueryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queries/src/test/org/apache/lucene/queries/CommonTermsQueryTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queries/src/test/org/apache/lucene/queries/CommonTermsQueryTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queries/src/test/org/apache/lucene/queries/CommonTermsQueryTest.java Tue Aug 13 04:06:18 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.queries;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
@@ -146,6 +147,8 @@ public class CommonTermsQueryTest extend
         left.add(new Term(_TestUtil.randomRealisticUnicodeString(r), _TestUtil
             .randomRealisticUnicodeString(r)));
       }
+      left.setHighFreqMinimumNumberShouldMatch(r.nextInt(4));
+      left.setLowFreqMinimumNumberShouldMatch(r.nextInt(4));
       
       r = new Random(seed);
       CommonTermsQuery right = new CommonTermsQuery(randomOccur(r),
@@ -155,6 +158,8 @@ public class CommonTermsQueryTest extend
         right.add(new Term(_TestUtil.randomRealisticUnicodeString(r), _TestUtil
             .randomRealisticUnicodeString(r)));
       }
+      right.setHighFreqMinimumNumberShouldMatch(r.nextInt(4));
+      right.setLowFreqMinimumNumberShouldMatch(r.nextInt(4));
       QueryUtils.checkEqual(left, right);
     }
   }
@@ -200,7 +205,7 @@ public class CommonTermsQueryTest extend
       query.add(new Term("field", "world"));
       query.add(new Term("field", "universe"));
       query.add(new Term("field", "right"));
-      query.setMinimumNumberShouldMatch(0.5f);
+      query.setLowFreqMinimumNumberShouldMatch(0.5f);
       TopDocs search = s.search(query, 10);
       assertEquals(search.totalHits, 1);
       assertEquals("0", r.document(search.scoreDocs[0].doc).get("id"));
@@ -214,7 +219,7 @@ public class CommonTermsQueryTest extend
       query.add(new Term("field", "world"));
       query.add(new Term("field", "universe"));
       query.add(new Term("field", "right"));
-      query.setMinimumNumberShouldMatch(2.0f);
+      query.setLowFreqMinimumNumberShouldMatch(2.0f);
       TopDocs search = s.search(query, 10);
       assertEquals(search.totalHits, 1);
       assertEquals("0", r.document(search.scoreDocs[0].doc).get("id"));
@@ -229,7 +234,7 @@ public class CommonTermsQueryTest extend
       query.add(new Term("field", "world"));
       query.add(new Term("field", "universe"));
       query.add(new Term("field", "right"));
-      query.setMinimumNumberShouldMatch(0.49f);
+      query.setLowFreqMinimumNumberShouldMatch(0.49f);
       TopDocs search = s.search(query, 10);
       assertEquals(search.totalHits, 3);
       assertEquals("0", r.document(search.scoreDocs[0].doc).get("id"));
@@ -246,14 +251,37 @@ public class CommonTermsQueryTest extend
       query.add(new Term("field", "world"));
       query.add(new Term("field", "universe"));
       query.add(new Term("field", "right"));
-      query.setMinimumNumberShouldMatch(1.0f);
+      query.setLowFreqMinimumNumberShouldMatch(1.0f);
       TopDocs search = s.search(query, 10);
       assertEquals(search.totalHits, 3);
       assertEquals("0", r.document(search.scoreDocs[0].doc).get("id"));
       assertEquals("2", r.document(search.scoreDocs[1].doc).get("id"));
       assertEquals("3", r.document(search.scoreDocs[2].doc).get("id"));
+      assertTrue(search.scoreDocs[1].score > search.scoreDocs[2].score);
+    }
+    
+    {
+      CommonTermsQuery query = new CommonTermsQuery(Occur.SHOULD, Occur.SHOULD,
+          random().nextBoolean() ? 2.0f : 0.5f);
+      query.add(new Term("field", "is"));
+      query.add(new Term("field", "this"));
+      query.add(new Term("field", "end"));
+      query.add(new Term("field", "world"));
+      query.add(new Term("field", "universe"));
+      query.add(new Term("field", "right"));
+      query.setLowFreqMinimumNumberShouldMatch(1.0f);
+      query.setHighFreqMinimumNumberShouldMatch(4.0f);
+      TopDocs search = s.search(query, 10);
+      assertEquals(search.totalHits, 3);
+      assertEquals(search.scoreDocs[1].score, search.scoreDocs[2].score, 0.0f);
+      assertEquals("0", r.document(search.scoreDocs[0].doc).get("id"));
+      // doc 2 and 3 only get a score from low freq terms
+      assertEquals(
+          new HashSet<>(Arrays.asList("2", "3")),
+          new HashSet<>(Arrays.asList(
+              r.document(search.scoreDocs[1].doc).get("id"),
+              r.document(search.scoreDocs[2].doc).get("id"))));
     }
-   
     r.close();
     w.close();
     dir.close();

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/build.xml?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/build.xml (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/build.xml Tue Aug 13 04:06:18 2013
@@ -152,4 +152,6 @@ import org.apache.lucene.queryparser.fle
     </sequential>
   </macrodef>
 
+  <target name="regenerate" depends="javacc"/>
+
 </project>

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/CharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/CharStream.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/CharStream.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/CharStream.java Tue Aug 13 04:06:18 2013
@@ -112,4 +112,4 @@ interface CharStream {
   void Done();
 
 }
-/* JavaCC - OriginalChecksum=c847dd1920bf7901125a7244125682ad (do not edit this line) */
+/* JavaCC - OriginalChecksum=30b94cad7b10d0d81e3a59a1083939d0 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/ParseException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/ParseException.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/ParseException.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/ParseException.java Tue Aug 13 04:06:18 2013
@@ -184,4 +184,4 @@ public class ParseException extends Exce
    }
 
 }
-/* JavaCC - OriginalChecksum=61602edcb3a15810cbc58f5593eba40d (do not edit this line) */
+/* JavaCC - OriginalChecksum=b187d97d5bb75c3fc63d642c1c26ac6e (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/Token.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/Token.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/Token.java Tue Aug 13 04:06:18 2013
@@ -128,4 +128,4 @@ public class Token implements java.io.Se
   }
 
 }
-/* JavaCC - OriginalChecksum=c1e1418b35aa9e47ef8dc98b87423d70 (do not edit this line) */
+/* JavaCC - OriginalChecksum=405bb5d2fcd84e94ac1c8f0b12c1f914 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/TokenMgrError.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/TokenMgrError.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/TokenMgrError.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/TokenMgrError.java Tue Aug 13 04:06:18 2013
@@ -144,4 +144,4 @@ public class TokenMgrError extends Error
     this(LexicalError(EOFSeen, lexState, errorLine, errorColumn, errorAfter, curChar), reason);
   }
 }
-/* JavaCC - OriginalChecksum=0c275864a1972d9a01601ab81426872d (do not edit this line) */
+/* JavaCC - OriginalChecksum=f433e1a52b8eadbf12f3fbbbf87fd140 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/CharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/CharStream.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/CharStream.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/CharStream.java Tue Aug 13 04:06:18 2013
@@ -112,4 +112,4 @@ interface CharStream {
   void Done();
 
 }
-/* JavaCC - OriginalChecksum=c95f1720d9b38046dc5d294b741c44cb (do not edit this line) */
+/* JavaCC - OriginalChecksum=53b2ec7502d50e2290e86187a6c01270 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/ParseException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/ParseException.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/ParseException.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/ParseException.java Tue Aug 13 04:06:18 2013
@@ -187,4 +187,4 @@ public class ParseException extends Quer
    }
 
 }
-/* JavaCC - OriginalChecksum=81401c29cf6f9909761c636b4778ccc0 (do not edit this line) */
+/* JavaCC - OriginalChecksum=4263a02db9988d7a863aa97ad2f6dc67 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/Token.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/Token.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/Token.java Tue Aug 13 04:06:18 2013
@@ -128,4 +128,4 @@ public class Token implements java.io.Se
   }
 
 }
-/* JavaCC - OriginalChecksum=30bbd23e0dec26f141130dc62a4f6e9d (do not edit this line) */
+/* JavaCC - OriginalChecksum=ea8b1e55950603be28e2f63dcd544ab4 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/TokenMgrError.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/TokenMgrError.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/TokenMgrError.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/parser/TokenMgrError.java Tue Aug 13 04:06:18 2013
@@ -144,4 +144,4 @@ public class TokenMgrError extends Error
     this(LexicalError(EOFSeen, lexState, errorLine, errorColumn, errorAfter, curChar), reason);
   }
 }
-/* JavaCC - OriginalChecksum=3ca7fbf7de9f2424b131a5499b0a78d0 (do not edit this line) */
+/* JavaCC - OriginalChecksum=be88283d82a985d82a34dda46bcf42d5 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/CharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/CharStream.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/CharStream.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/CharStream.java Tue Aug 13 04:06:18 2013
@@ -112,4 +112,4 @@ interface CharStream {
   void Done();
 
 }
-/* JavaCC - OriginalChecksum=5ca20c9145f29a0f8909470a7f949fe4 (do not edit this line) */
+/* JavaCC - OriginalChecksum=242ae59b965491e225a44534cbc73b42 (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/ParseException.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/ParseException.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/ParseException.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/ParseException.java Tue Aug 13 04:06:18 2013
@@ -184,4 +184,4 @@ public class ParseException extends Exce
    }
 
 }
-/* JavaCC - OriginalChecksum=be6f55e3bf157e8c96b4c06cca5ec81b (do not edit this line) */
+/* JavaCC - OriginalChecksum=bd8163f41bf2fd1bb00f025fce3dcaaf (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/Token.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/Token.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/Token.java Tue Aug 13 04:06:18 2013
@@ -128,4 +128,4 @@ public class Token implements java.io.Se
   }
 
 }
-/* JavaCC - OriginalChecksum=db38f23b3674db52ff034369707a0ac3 (do not edit this line) */
+/* JavaCC - OriginalChecksum=f2df701e24da1cf2d025118ce6efdd2f (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/TokenMgrError.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/TokenMgrError.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/TokenMgrError.java (original)
+++ lucene/dev/branches/lucene3069/lucene/queryparser/src/java/org/apache/lucene/queryparser/surround/parser/TokenMgrError.java Tue Aug 13 04:06:18 2013
@@ -144,4 +144,4 @@ public class TokenMgrError extends Error
     this(LexicalError(EOFSeen, lexState, errorLine, errorColumn, errorAfter, curChar), reason);
   }
 }
-/* JavaCC - OriginalChecksum=dcdd5ccde13b91bcd8f76a86ca618852 (do not edit this line) */
+/* JavaCC - OriginalChecksum=8c69a370d9a9893140562c8bb911678c (do not edit this line) */

Modified: lucene/dev/branches/lucene3069/lucene/replicator/src/java/org/apache/lucene/replicator/IndexAndTaxonomyRevision.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/replicator/src/java/org/apache/lucene/replicator/IndexAndTaxonomyRevision.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/replicator/src/java/org/apache/lucene/replicator/IndexAndTaxonomyRevision.java (original)
+++ lucene/dev/branches/lucene3069/lucene/replicator/src/java/org/apache/lucene/replicator/IndexAndTaxonomyRevision.java Tue Aug 13 04:06:18 2013
@@ -78,15 +78,14 @@ public class IndexAndTaxonomyRevision im
     @Override
     protected IndexWriterConfig createIndexWriterConfig(OpenMode openMode) {
       IndexWriterConfig conf = super.createIndexWriterConfig(openMode);
-      conf.setIndexDeletionPolicy(new SnapshotDeletionPolicy(conf.getIndexDeletionPolicy()));
+      sdp = new SnapshotDeletionPolicy(conf.getIndexDeletionPolicy());
+      conf.setIndexDeletionPolicy(sdp);
       return conf;
     }
     
     @Override
     protected IndexWriter openIndexWriter(Directory directory, IndexWriterConfig config) throws IOException {
       writer = super.openIndexWriter(directory, config);
-      // must set it here because IndexWriter clones the config
-      sdp = (SnapshotDeletionPolicy) writer.getConfig().getIndexDeletionPolicy();
       return writer;
     }
     

Modified: lucene/dev/branches/lucene3069/lucene/site/xsl/index.xsl
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/site/xsl/index.xsl?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/site/xsl/index.xsl (original)
+++ lucene/dev/branches/lucene3069/lucene/site/xsl/index.xsl Tue Aug 13 04:06:18 2013
@@ -36,7 +36,15 @@
       </head>
       <body>
         <div><a href="http://lucene.apache.org/core/"><img src="lucene_green_300.gif" title="Apache Lucene Logo" alt="Lucene" border="0"/></a></div>
-        <h1><xsl:text>Apache Lucene </xsl:text><xsl:value-of select="$version"/><xsl:text> Documentation</xsl:text></h1>
+        <h1>
+          <xsl:text>Apache Lucene</xsl:text>
+          <span style="vertical-align: top; font-size: x-small">
+            <xsl:text>TM</xsl:text>
+          </span>
+          <xsl:text> </xsl:text>
+          <xsl:value-of select="$version"/>
+          <xsl:text> Documentation</xsl:text>
+        </h1>
         <p>Lucene is a Java full-text search engine. Lucene is not a complete application, 
         but rather a code library and API that can easily be used to add search capabilities
         to applications.</p>
@@ -71,7 +79,6 @@
             <li><a href="core/org/apache/lucene/search/package-summary.html#package_description">Search and Scoring in Lucene</a>: Introduction to how Lucene scores documents.</li>
             <li><a href="core/org/apache/lucene/search/similarities/TFIDFSimilarity.html">Classic Scoring Formula</a>: Formula of Lucene's classic <a href="http://en.wikipedia.org/wiki/Vector_Space_Model">Vector Space</a> implementation. (look <a href="core/org/apache/lucene/search/similarities/package-summary.html#package_description">here</a> for other models)</li>
             <li><a href="queryparser/org/apache/lucene/queryparser/classic/package-summary.html#package_description">Classic QueryParser Syntax</a>: Overview of the Classic QueryParser's syntax and features.</li>
-            <li><a href="facet/org/apache/lucene/facet/doc-files/userguide.html">Facet User Guide</a>: User's Guide to implementing <a href="http://en.wikipedia.org/wiki/Faceted_search">Faceted search</a>.</li>
           </ul>
         <h2>API Javadocs</h2>
         <xsl:call-template name="modules"/>

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java Tue Aug 13 04:06:18 2013
@@ -102,11 +102,20 @@ public abstract class SpatialStrategy {
   public abstract Field[] createIndexableFields(Shape shape);
 
   /**
+   * See {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point, double)} called with
+   * a multiplier of 1.0 (i.e. units of degrees).
+   */
+  public ValueSource makeDistanceValueSource(Point queryPoint) {
+    return makeDistanceValueSource(queryPoint, 1.0);
+  }
+
+  /**
    * Make a ValueSource returning the distance between the center of the
    * indexed shape and {@code queryPoint}.  If there are multiple indexed shapes
-   * then the closest one is chosen.
+   * then the closest one is chosen. The result is multiplied by {@code multiplier}, which
+   * conveniently is used to get the desired units.
    */
-  public abstract ValueSource makeDistanceValueSource(Point queryPoint);
+  public abstract ValueSource makeDistanceValueSource(Point queryPoint, double multiplier);
 
   /**
    * Make a Query based principally on {@link org.apache.lucene.spatial.query.SpatialOperation}
@@ -139,7 +148,7 @@ public abstract class SpatialStrategy {
 
   /**
    * Returns a ValueSource with values ranging from 1 to 0, depending inversely
-   * on the distance from {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}.
+   * on the distance from {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}.
    * The formula is {@code c/(d + c)} where 'd' is the distance and 'c' is
    * one tenth the distance to the farthest edge from the center. Thus the
    * scores will be 1 for indexed points at the center of the query shape and as
@@ -151,7 +160,7 @@ public abstract class SpatialStrategy {
         ctx.makePoint(bbox.getMinX(), bbox.getMinY()), bbox.getMaxX(), bbox.getMaxY());
     double distToEdge = diagonalDist * 0.5;
     float c = (float)distToEdge * 0.1f;//one tenth
-    return new ReciprocalFloatFunction(makeDistanceValueSource(queryShape.getCenter()), 1f, c, c);
+    return new ReciprocalFloatFunction(makeDistanceValueSource(queryShape.getCenter(), 1.0), 1f, c, c);
   }
 
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java Tue Aug 13 04:06:18 2013
@@ -135,9 +135,9 @@ public class BBoxStrategy extends Spatia
   //---------------------------------
 
   @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint) {
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
     return new BBoxSimilarityValueSource(
-        this, new DistanceSimilarity(this.getSpatialContext(), queryPoint));
+        this, new DistanceSimilarity(this.getSpatialContext(), queryPoint, multiplier));
   }
 
   public ValueSource makeBBoxAreaSimilarityValueSource(Rectangle queryBox) {

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/DistanceSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/DistanceSimilarity.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/DistanceSimilarity.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/DistanceSimilarity.java Tue Aug 13 04:06:18 2013
@@ -30,13 +30,15 @@ import org.apache.lucene.search.Explanat
  */
 public class DistanceSimilarity implements BBoxSimilarity {
   private final Point queryPoint;
+  private final double multiplier;
   private final DistanceCalculator distCalc;
   private final double nullValue;
 
-  public DistanceSimilarity(SpatialContext ctx, Point queryPoint) {
+  public DistanceSimilarity(SpatialContext ctx, Point queryPoint, double multiplier) {
     this.queryPoint = queryPoint;
+    this.multiplier = multiplier;
     this.distCalc = ctx.getDistCalc();
-    this.nullValue = (ctx.isGeo() ? 180 : Double.MAX_VALUE);
+    this.nullValue = (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE);
   }
 
   @Override
@@ -45,14 +47,43 @@ public class DistanceSimilarity implemen
     if (indexRect == null) {
       score = nullValue;
     } else {
-      score = distCalc.distance(queryPoint, indexRect.getCenter());
+      score = distCalc.distance(queryPoint, indexRect.getCenter()) * multiplier;
     }
     if (exp != null) {
       exp.setValue((float)score);
       exp.setDescription(this.getClass().getSimpleName());
-      exp.addDetail(new Explanation(-1f,""+queryPoint));
+      exp.addDetail(new Explanation(-1f, "" + queryPoint));
       exp.addDetail(new Explanation(-1f,""+indexRect));
+      exp.addDetail(new Explanation((float)multiplier,"multiplier"));
     }
     return score;
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    DistanceSimilarity that = (DistanceSimilarity) o;
+
+    if (Double.compare(that.multiplier, multiplier) != 0) return false;
+    if (Double.compare(that.nullValue, nullValue) != 0) return false;
+    if (!distCalc.equals(that.distCalc)) return false;
+    if (!queryPoint.equals(that.queryPoint)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result;
+    long temp;
+    result = queryPoint.hashCode();
+    temp = Double.doubleToLongBits(multiplier);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    result = 31 * result + distCalc.hashCode();
+    temp = Double.doubleToLongBits(nullValue);
+    result = 31 * result + (int) (temp ^ (temp >>> 32));
+    return result;
+  }
 }

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java Tue Aug 13 04:06:18 2013
@@ -107,6 +107,14 @@ public abstract class AbstractVisitingPr
     this depth.  It would be nice if termsEnum knew how many terms
     start with the current term without having to repeatedly next() & test to find out.
 
+  * Perhaps don't do intermediate seek()'s to cells above detailLevel that have Intersects
+    relation because we won't be collecting those docs any way.  However seeking
+    does act as a short-circuit.  So maybe do some percent of the time or when the level
+    is above some threshold.
+
+  * Each shape.relate(otherShape) result could be cached since much of the same relations
+    will be invoked when multiple segments are involved.
+
   */
 
     protected final boolean hasIndexedLeaves;//if false then we can skip looking for them
@@ -171,11 +179,11 @@ public abstract class AbstractVisitingPr
         int compare = termsEnum.getComparator().compare(thisTerm, curVNodeTerm);
         if (compare > 0) {
           // leap frog (termsEnum is beyond where we would otherwise seek)
-          assert ! context.reader().terms(fieldName).iterator(null).seekExact(curVNodeTerm, false) : "should be absent";
+          assert ! context.reader().terms(fieldName).iterator(null).seekExact(curVNodeTerm) : "should be absent";
         } else {
           if (compare < 0) {
             // Seek !
-            TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(curVNodeTerm, true);
+            TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(curVNodeTerm);
             if (seekStatus == TermsEnum.SeekStatus.END)
               break; // all done
             thisTerm = termsEnum.term();
@@ -339,7 +347,7 @@ public abstract class AbstractVisitingPr
   }//class VisitorTemplate
 
   /**
-   * A Visitor Cell/Cell found via the query shape for {@link VisitorTemplate}.
+   * A visitor node/cell found via the query shape for {@link VisitorTemplate}.
    * Sometimes these are reset(cell). It's like a LinkedList node but forms a
    * tree.
    *

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java Tue Aug 13 04:06:18 2013
@@ -131,10 +131,11 @@ public class ContainsPrefixTreeFilter ex
 
     private boolean seekExact(Cell cell) throws IOException {
       assert new BytesRef(cell.getTokenBytes()).compareTo(termBytes) > 0;
-
       termBytes.bytes = cell.getTokenBytes();
       termBytes.length = termBytes.bytes.length;
-      return termsEnum.seekExact(termBytes, cell.getLevel() <= 2);
+      if (termsEnum == null)
+        return false;
+      return termsEnum.seekExact(termBytes);
     }
 
     private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
@@ -150,6 +151,8 @@ public class ContainsPrefixTreeFilter ex
       assert ! leafCell.equals(lastLeaf);//don't call for same leaf again
       lastLeaf = leafCell;
 
+      if (termsEnum == null)
+        return null;
       BytesRef nextTerm = termsEnum.next();
       if (nextTerm == null) {
         termsEnum = null;//signals all done

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Tue Aug 13 04:06:18 2013
@@ -56,7 +56,7 @@ import java.util.concurrent.ConcurrentHa
  * <li>Only {@link org.apache.lucene.spatial.query.SpatialOperation#Intersects}
  * is supported.  If only points are indexed then this is effectively equivalent
  * to IsWithin.</li>
- * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}
+ * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}
  * even for multi-valued data, so long as the indexed data is all points; the
  * behavior is undefined otherwise.  However, <em>it will likely be removed in
  * the future</em> in lieu of using another strategy with a more scalable
@@ -182,7 +182,7 @@ public abstract class PrefixTreeStrategy
   }
 
   @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint) {
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
     PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );
     if( p == null ) {
       synchronized (this) {//double checked locking idiom is okay since provider is threadsafe
@@ -194,7 +194,7 @@ public abstract class PrefixTreeStrategy
       }
     }
 
-    return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint);
+    return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint, multiplier);
   }
 
   public SpatialPrefixTree getGrid() {

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java Tue Aug 13 04:06:18 2013
@@ -38,14 +38,17 @@ import java.util.Map;
  */
 public class ShapeFieldCacheDistanceValueSource extends ValueSource {
 
-  private final ShapeFieldCacheProvider<Point> provider;
   private final SpatialContext ctx;
   private final Point from;
+  private final ShapeFieldCacheProvider<Point> provider;
+  private final double multiplier;
 
-  public ShapeFieldCacheDistanceValueSource(SpatialContext ctx, ShapeFieldCacheProvider<Point> provider, Point from) {
+  public ShapeFieldCacheDistanceValueSource(SpatialContext ctx,
+      ShapeFieldCacheProvider<Point> provider, Point from, double multiplier) {
     this.ctx = ctx;
     this.from = from;
     this.provider = provider;
+    this.multiplier = multiplier;
   }
 
   @Override
@@ -60,7 +63,7 @@ public class ShapeFieldCacheDistanceValu
           provider.getCache(readerContext.reader());
       private final Point from = ShapeFieldCacheDistanceValueSource.this.from;
       private final DistanceCalculator calculator = ctx.getDistCalc();
-      private final double nullValue = (ctx.isGeo() ? 180 : Double.MAX_VALUE);
+      private final double nullValue = (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE);
 
       @Override
       public float floatVal(int doc) {
@@ -69,13 +72,14 @@ public class ShapeFieldCacheDistanceValu
 
       @Override
       public double doubleVal(int doc) {
+
         List<Point> vals = cache.getShapes( doc );
         if( vals != null ) {
           double v = calculator.distance(from, vals.get(0));
           for( int i=1; i<vals.size(); i++ ) {
             v = Math.min(v, calculator.distance(from, vals.get(i)));
           }
-          return v;
+          return v * multiplier;
         }
         return nullValue;
       }
@@ -97,6 +101,7 @@ public class ShapeFieldCacheDistanceValu
     if (!ctx.equals(that.ctx)) return false;
     if (!from.equals(that.from)) return false;
     if (!provider.equals(that.provider)) return false;
+    if (multiplier != that.multiplier) return false;
 
     return true;
   }

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java Tue Aug 13 04:06:18 2013
@@ -39,13 +39,15 @@ public class DistanceValueSource extends
 
   private PointVectorStrategy strategy;
   private final Point from;
+  private final double multiplier;
 
   /**
    * Constructor.
    */
-  public DistanceValueSource(PointVectorStrategy strategy, Point from) {
+  public DistanceValueSource(PointVectorStrategy strategy, Point from, double multiplier) {
     this.strategy = strategy;
     this.from = from;
+    this.multiplier = multiplier;
   }
 
   /**
@@ -72,7 +74,8 @@ public class DistanceValueSource extends
 
       private final Point from = DistanceValueSource.this.from;
       private final DistanceCalculator calculator = strategy.getSpatialContext().getDistCalc();
-      private final double nullValue = (strategy.getSpatialContext().isGeo() ? 180 : Double.MAX_VALUE);
+      private final double nullValue =
+          (strategy.getSpatialContext().isGeo() ? 180 * multiplier : Double.MAX_VALUE);
 
       @Override
       public float floatVal(int doc) {
@@ -84,7 +87,7 @@ public class DistanceValueSource extends
         // make sure it has minX and area
         if (validX.get(doc)) {
           assert validY.get(doc);
-          return calculator.distance(from, ptX.get(doc), ptY.get(doc));
+          return calculator.distance(from, ptX.get(doc), ptY.get(doc)) * multiplier;
         }
         return nullValue;
       }
@@ -105,6 +108,7 @@ public class DistanceValueSource extends
 
     if (!from.equals(that.from)) return false;
     if (!strategy.equals(that.strategy)) return false;
+    if (multiplier != that.multiplier) return false;
 
     return true;
   }

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java Tue Aug 13 04:06:18 2013
@@ -120,8 +120,8 @@ public class PointVectorStrategy extends
   }
 
   @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint) {
-    return new DistanceValueSource(this, queryPoint);
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    return new DistanceValueSource(this, queryPoint, multiplier);
   }
 
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java Tue Aug 13 04:06:18 2013
@@ -34,6 +34,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 public class DistanceStrategyTest extends StrategyTestCase {
@@ -121,6 +122,11 @@ public class DistanceStrategyTest extend
 
   void checkDistValueSource(String ptStr, float... distances) throws IOException {
     Point pt = (Point) ctx.readShape(ptStr);
-    checkValueSource(strategy.makeDistanceValueSource(pt), distances, 1.0e-4f);
+    float multiplier = random().nextFloat() * 100f;
+    float[] dists2 = Arrays.copyOf(distances, distances.length);
+    for (int i = 0; i < dists2.length; i++) {
+      dists2[i] *= multiplier;
+    }
+    checkValueSource(strategy.makeDistanceValueSource(pt, multiplier), dists2, 1.0e-3f);
   }
 }

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java Tue Aug 13 04:06:18 2013
@@ -162,7 +162,8 @@ public class SpatialExample extends Luce
     //--Match all, order by distance ascending
     {
       Point pt = ctx.makePoint(60, -50);
-      ValueSource valueSource = strategy.makeDistanceValueSource(pt);//the distance (in degrees)
+      double degToKm = DistanceUtils.degrees2Dist(1, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+      ValueSource valueSource = strategy.makeDistanceValueSource(pt, degToKm);//the distance (in km)
       Sort distSort = new Sort(valueSource.getSortField(false)).rewrite(indexSearcher);//false=asc dist
       TopDocs docs = indexSearcher.search(new MatchAllDocsQuery(), 10, distSort);
       assertDocMatchedIds(indexSearcher, docs, 4, 20, 2);

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java Tue Aug 13 04:06:18 2013
@@ -58,6 +58,8 @@ public abstract class SpatialTestCase ex
 
     directory = newDirectory();
     indexWriter = new RandomIndexWriter(random(),directory);
+    indexReader = indexWriter.getReader();
+    indexSearcher = newSearcher(indexReader);
   }
 
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java Tue Aug 13 04:06:18 2013
@@ -186,6 +186,13 @@ public class SpatialOpRecursivePrefixTre
   }
 
   private void doTest(final SpatialOperation operation) throws IOException {
+    //first show that when there's no data, a query will result in no results
+    {
+      Query query = strategy.makeQuery(new SpatialArgs(operation, randomRectangle()));
+      SearchResults searchResults = executeQuery(query, 1);
+      assertEquals(0, searchResults.numFound);
+    }
+
     final boolean biasContains = (operation == SpatialOperation.Contains);
 
     Map<String, Shape> indexedShapes = new LinkedHashMap<String, Shape>();

Modified: lucene/dev/branches/lucene3069/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java (original)
+++ lucene/dev/branches/lucene3069/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java Tue Aug 13 04:06:18 2013
@@ -525,7 +525,7 @@ public class SpellChecker implements jav
   
           if (!isEmpty) {
             for (TermsEnum te : termsEnums) {
-              if (te.seekExact(currentTerm, false)) {
+              if (te.seekExact(currentTerm)) {
                 continue terms;
               }
             }