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/12/01 18:25:47 UTC

svn commit: r1642718 [5/12] - in /lucene/dev/branches/lucene2878: ./ dev-tools/ dev-tools/eclipse/dot.settings/ dev-tools/idea/.idea/ dev-tools/idea/lucene/benchmark/src/ dev-tools/idea/lucene/highlighter/ dev-tools/maven/ dev-tools/maven/solr/webapp/ ...

Modified: lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/highlighter/src/test/org/apache/lucene/search/highlight/TokenSourcesTest.java Mon Dec  1 17:25:39 2014
@@ -19,6 +19,8 @@ package org.apache.lucene.search.highlig
 
 import java.io.IOException;
 
+import com.carrotsearch.randomizedtesting.annotations.Repeat;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
 import org.apache.lucene.analysis.CannedTokenStream;
 import org.apache.lucene.analysis.Token;
 import org.apache.lucene.analysis.TokenStream;
@@ -30,6 +32,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.BaseTermVectorsFormatTestCase;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -44,10 +47,10 @@ import org.apache.lucene.search.spans.Sp
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
 
 // LUCENE-2874
-public class TokenSourcesTest extends LuceneTestCase {
+public class TokenSourcesTest extends BaseTokenStreamTestCase {
   private static final String FIELD = "text";
 
   private static final class OverlappingTokenStream extends TokenStream {
@@ -121,8 +124,7 @@ public class TokenSourcesTest extends Lu
           new QueryScorer(query));
       final TokenStream tokenStream = TokenSources
           .getTokenStream(
-              indexReader.getTermVector(0, FIELD),
-              false);
+              indexReader.getTermVector(0, FIELD));
       assertEquals("<B>the fox</B> did not jump",
           highlighter.getBestFragment(tokenStream, TEXT));
     } finally {
@@ -166,8 +168,7 @@ public class TokenSourcesTest extends Lu
           new QueryScorer(query));
       final TokenStream tokenStream = TokenSources
           .getTokenStream(
-              indexReader.getTermVector(0, FIELD),
-              false);
+              indexReader.getTermVector(0, FIELD));
       assertEquals("<B>the fox</B> did not jump",
           highlighter.getBestFragment(tokenStream, TEXT));
     } finally {
@@ -210,8 +211,7 @@ public class TokenSourcesTest extends Lu
           new QueryScorer(phraseQuery));
       final TokenStream tokenStream = TokenSources
           .getTokenStream(
-              indexReader.getTermVector(0, FIELD),
-              false);
+              indexReader.getTermVector(0, FIELD));
       assertEquals("<B>the fox</B> did not jump",
           highlighter.getBestFragment(tokenStream, TEXT));
     } finally {
@@ -254,8 +254,7 @@ public class TokenSourcesTest extends Lu
           new QueryScorer(phraseQuery));
       final TokenStream tokenStream = TokenSources
           .getTokenStream(
-              indexReader.getTermVector(0, FIELD),
-              false);
+              indexReader.getTermVector(0, FIELD));
       assertEquals("<B>the fox</B> did not jump",
           highlighter.getBestFragment(tokenStream, TEXT));
     } finally {
@@ -284,8 +283,7 @@ public class TokenSourcesTest extends Lu
     try {
       assertEquals(1, indexReader.numDocs());
       TokenSources.getTokenStream(
-              indexReader.getTermVector(0, FIELD),
-              false);
+              indexReader.getTermVector(0, FIELD));
       fail("TokenSources.getTokenStream should throw IllegalArgumentException if term vector has no offsets");
     }
     catch (IllegalArgumentException e) {
@@ -335,27 +333,98 @@ public class TokenSourcesTest extends Lu
     writer.close();
     assertEquals(1, reader.numDocs());
 
-    for(int i=0;i<2;i++) {
-      // Do this twice, once passing true and then passing
-      // false: they are entirely different code paths
-      // under-the-hood:
-      TokenStream ts = TokenSources.getTokenStream(reader.getTermVectors(0).terms("field"), i == 0);
-
-      CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
-      PositionIncrementAttribute posIncAtt = ts.getAttribute(PositionIncrementAttribute.class);
-      OffsetAttribute offsetAtt = ts.getAttribute(OffsetAttribute.class);
-      PayloadAttribute payloadAtt = ts.getAttribute(PayloadAttribute.class);
-
-      for(Token token : tokens) {
-        assertTrue(ts.incrementToken());
-        assertEquals(token.toString(), termAtt.toString());
-        assertEquals(token.getPositionIncrement(), posIncAtt.getPositionIncrement());
-        assertEquals(token.getPayload(), payloadAtt.getPayload());
-        assertEquals(token.startOffset(), offsetAtt.startOffset());
-        assertEquals(token.endOffset(), offsetAtt.endOffset());
+    TokenStream ts = TokenSources.getTokenStream(reader.getTermVectors(0).terms("field"));
+
+    CharTermAttribute termAtt = ts.getAttribute(CharTermAttribute.class);
+    PositionIncrementAttribute posIncAtt = ts.getAttribute(PositionIncrementAttribute.class);
+    OffsetAttribute offsetAtt = ts.getAttribute(OffsetAttribute.class);
+    PayloadAttribute payloadAtt = ts.addAttribute(PayloadAttribute.class);
+
+    ts.reset();
+    for(Token token : tokens) {
+      assertTrue(ts.incrementToken());
+      assertEquals(token.toString(), termAtt.toString());
+      assertEquals(token.getPositionIncrement(), posIncAtt.getPositionIncrement());
+      assertEquals(token.getPayload(), payloadAtt.getPayload());
+      assertEquals(token.startOffset(), offsetAtt.startOffset());
+      assertEquals(token.endOffset(), offsetAtt.endOffset());
+    }
+
+    assertFalse(ts.incrementToken());
+
+    reader.close();
+    dir.close();
+  }
+
+  @Repeat(iterations = 10)
+  //@Seed("947083AB20AB2D4F")
+  public void testRandomizedRoundTrip() throws Exception {
+    final int distinct = TestUtil.nextInt(random(), 1, 10);
+
+    String[] terms = new String[distinct];
+    BytesRef[] termBytes = new BytesRef[distinct];
+    for (int i = 0; i < distinct; ++i) {
+      terms[i] = TestUtil.randomRealisticUnicodeString(random());
+      termBytes[i] = new BytesRef(terms[i]);
+    }
+
+    final BaseTermVectorsFormatTestCase.RandomTokenStream rTokenStream =
+        new BaseTermVectorsFormatTestCase.RandomTokenStream(TestUtil.nextInt(random(), 1, 10), terms, termBytes, false);
+    //check to see if the token streams might have non-deterministic testable result
+    final boolean storeTermVectorPositions = random().nextBoolean();
+    final int[] startOffsets = rTokenStream.getStartOffsets();
+    final int[] positionsIncrements = rTokenStream.getPositionsIncrements();
+    for (int i = 1; i < positionsIncrements.length; i++) {
+      if (storeTermVectorPositions && positionsIncrements[i] != 0) {
+        continue;
+      }
+      //TODO should RandomTokenStream ensure endOffsets for tokens at same position and same startOffset are greater
+      // than previous token's endOffset?  That would increase the testable possibilities.
+      if (startOffsets[i] == startOffsets[i-1]) {
+        if (VERBOSE)
+          System.out.println("Skipping test because can't easily validate random token-stream is correct.");
+        return;
       }
+    }
+
+    //sanity check itself
+    assertTokenStreamContents(rTokenStream,
+        rTokenStream.getTerms(), rTokenStream.getStartOffsets(), rTokenStream.getEndOffsets(),
+        rTokenStream.getPositionsIncrements());
+
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    FieldType myFieldType = new FieldType(TextField.TYPE_NOT_STORED);
+    myFieldType.setStoreTermVectors(true);
+    myFieldType.setStoreTermVectorOffsets(true);
+    myFieldType.setStoreTermVectorPositions(storeTermVectorPositions);
+    //payloads require positions; it will throw an error otherwise
+    myFieldType.setStoreTermVectorPayloads(storeTermVectorPositions && random().nextBoolean());
+
+    Document doc = new Document();
+    doc.add(new Field("field", rTokenStream, myFieldType));
+    writer.addDocument(doc);
 
-      assertFalse(ts.incrementToken());
+    IndexReader reader = writer.getReader();
+    writer.close();
+    assertEquals(1, reader.numDocs());
+
+    TokenStream vectorTokenStream = TokenSources.getTokenStream(reader.getTermVectors(0).terms("field"));
+
+    //sometimes check payloads
+    PayloadAttribute payloadAttribute = null;
+    if (myFieldType.storeTermVectorPayloads() && usually()) {
+      payloadAttribute = vectorTokenStream.addAttribute(PayloadAttribute.class);
+    }
+    assertTokenStreamContents(vectorTokenStream,
+        rTokenStream.getTerms(), rTokenStream.getStartOffsets(), rTokenStream.getEndOffsets(),
+        myFieldType.storeTermVectorPositions() ? rTokenStream.getPositionsIncrements() : null);
+    //test payloads
+    if (payloadAttribute != null) {
+      vectorTokenStream.reset();
+      for (int i = 0; vectorTokenStream.incrementToken(); i++) {
+        assertEquals(rTokenStream.getPayloads()[i], payloadAttribute.getPayload());
+      }
     }
 
     reader.close();

Modified: lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java (original)
+++ lucene/dev/branches/lucene2878/lucene/join/src/java/org/apache/lucene/search/join/BitDocIdSetCachingWrapperFilter.java Mon Dec  1 17:25:39 2014
@@ -25,6 +25,7 @@ import org.apache.lucene.search.CachingW
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.FilterCachingPolicy;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BitDocIdSet;
 
@@ -39,7 +40,7 @@ public class BitDocIdSetCachingWrapperFi
   /** Sole constructor. */
   public BitDocIdSetCachingWrapperFilter(Filter filter) {
     super();
-    this.filter = new CachingWrapperFilter(filter) {
+    this.filter = new CachingWrapperFilter(filter, FilterCachingPolicy.ALWAYS_CACHE) {
       @Override
       protected BitDocIdSet docIdSetToCache(DocIdSet docIdSet, LeafReader reader) throws IOException {
         if (docIdSet == null || docIdSet instanceof BitDocIdSet) {

Modified: lucene/dev/branches/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/branches/lucene2878/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Mon Dec  1 17:25:39 2014
@@ -72,9 +72,9 @@ import java.util.NoSuchElementException;
 
 /**
  * High-performance single-document main memory Apache Lucene fulltext search index. 
- * 
- * <h4>Overview</h4>
- * 
+ * <p>
+ * <b>Overview</b>
+ * <p>
  * This class is a replacement/substitute for a large subset of
  * {@link RAMDirectory} functionality. It is designed to
  * enable maximum efficiency for on-the-fly matchmaking combining structured and 
@@ -119,9 +119,9 @@ import java.util.NoSuchElementException;
  * <a target="_blank" 
  * href="http://www.tbray.org/ongoing/When/200x/2003/07/30/OnSearchTOC">On Search, the Series</a>.
  * 
- * 
- * <h4>Example Usage</h4> 
- * 
+ * <p>
+ * <b>Example Usage</b> 
+ * <p>
  * <pre class="prettyprint">
  * Analyzer analyzer = new SimpleAnalyzer(version);
  * MemoryIndex index = new MemoryIndex();
@@ -137,29 +137,29 @@ import java.util.NoSuchElementException;
  * System.out.println("indexData=" + index.toString());
  * </pre>
  * 
- * 
- * <h4>Example XQuery Usage</h4> 
+ * <p>
+ * <b>Example XQuery Usage</b> 
  * 
  * <pre class="prettyprint">
  * (: An XQuery that finds all books authored by James that have something to do with "salmon fishing manuals", sorted by relevance :)
  * declare namespace lucene = "java:nux.xom.pool.FullTextUtil";
  * declare variable $query := "+salmon~ +fish* manual~"; (: any arbitrary Lucene query can go here :)
  * 
- * for $book in /books/book[author="James" and lucene:match(abstract, $query) > 0.0]
+ * for $book in /books/book[author="James" and lucene:match(abstract, $query) &gt; 0.0]
  * let $score := lucene:match($book/abstract, $query)
  * order by $score descending
  * return $book
  * </pre>
  * 
- * 
- * <h4>Thread safety guarantees</h4>
- *
+ * <p>
+ * <b>Thread safety guarantees</b>
+ * <p>
  * MemoryIndex is not normally thread-safe for adds or queries.  However, queries
  * are thread-safe after {@code freeze()} has been called.
  *
- *
- * <h4>Performance Notes</h4>
- * 
+ * <p>
+ * <b>Performance Notes</b>
+ * <p>
  * Internally there's a new data structure geared towards efficient indexing 
  * and searching, plus the necessary support code to seamlessly plug into the Lucene 
  * framework.
@@ -187,7 +187,7 @@ import java.util.NoSuchElementException;
  */
 public class MemoryIndex {
 
-  /** info for each field: Map<String fieldName, Info field> */
+  /** info for each field: Map&lt;String fieldName, Info field&gt; */
   private final HashMap<String,Info> fields = new HashMap<>();
   
   /** fields sorted ascending by fieldName; lazily computed on demand */
@@ -687,8 +687,8 @@ public class MemoryIndex {
   private static final class Info {
     
     /**
-     * Term strings and their positions for this field: Map <String
-     * termText, ArrayIntList positions>
+     * Term strings and their positions for this field: Map &lt;String
+     * termText, ArrayIntList positions&gt;
      */
     private final BytesRefHash terms; 
     

Modified: lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java (original)
+++ lucene/dev/branches/lucene2878/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java Mon Dec  1 17:25:39 2014
@@ -194,14 +194,14 @@ public class DocTermOrds implements Acco
 
   /** Inverts only terms starting w/ prefix, and only terms
    *  whose docFreq (not taking deletions into account) is
-   *  <=  maxTermDocFreq */
+   *  &lt;=  maxTermDocFreq */
   public DocTermOrds(LeafReader reader, Bits liveDocs, String field, BytesRef termPrefix, int maxTermDocFreq) throws IOException {
     this(reader, liveDocs, field, termPrefix, maxTermDocFreq, DEFAULT_INDEX_INTERVAL_BITS);
   }
 
   /** Inverts only terms starting w/ prefix, and only terms
    *  whose docFreq (not taking deletions into account) is
-   *  <=  maxTermDocFreq, with a custom indexing interval
+   *  &lt;=  maxTermDocFreq, with a custom indexing interval
    *  (default is every 128nd term). */
   public DocTermOrds(LeafReader reader, Bits liveDocs, String field, BytesRef termPrefix, int maxTermDocFreq, int indexIntervalBits) throws IOException {
     this(field, maxTermDocFreq, indexIntervalBits);

Modified: lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java (original)
+++ lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/index/sorter/TestBlockJoinSorter.java Mon Dec  1 17:25:39 2014
@@ -36,6 +36,7 @@ import org.apache.lucene.search.CachingW
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.FilterCachingPolicy;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
@@ -50,7 +51,7 @@ public class TestBlockJoinSorter extends
   private static class FixedBitSetCachingWrapperFilter extends CachingWrapperFilter {
 
     public FixedBitSetCachingWrapperFilter(Filter filter) {
-      super(filter);
+      super(filter, FilterCachingPolicy.ALWAYS_CACHE);
     }
 
     @Override

Modified: lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheVsDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheVsDocValues.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheVsDocValues.java (original)
+++ lucene/dev/branches/lucene2878/lucene/misc/src/test/org/apache/lucene/uninverting/TestFieldCacheVsDocValues.java Mon Dec  1 17:25:39 2014
@@ -578,6 +578,6 @@ public class TestFieldCacheVsDocValues e
   
   protected boolean codecAcceptsHugeBinaryValues(String field) {
     String name = TestUtil.getDocValuesFormat(field);
-    return !(name.equals("Memory") || name.equals("Direct"));
+    return !(name.equals("Memory")); // Direct has a different type of limit
   }
 }

Modified: lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/BoostingQuery.java Mon Dec  1 17:25:39 2014
@@ -34,7 +34,8 @@ import org.apache.lucene.search.*;
  * multiplied by the supplied "boost" parameter, so this should be less than 1 to achieve a 
  * demoting effect
  * 
- * This code was originally made available here: [WWW] http://marc.theaimsgroup.com/?l=lucene-user&m=108058407130459&w=2
+ * This code was originally made available here: 
+ *   <a href="http://marc.theaimsgroup.com/?l=lucene-user&m=108058407130459&w=2">http://marc.theaimsgroup.com/?l=lucene-user&amp;m=108058407130459&amp;w=2</a>
  * and is documented here: http://wiki.apache.org/lucene-java/CommunityContributions
  */
 public class BoostingQuery extends Query {

Modified: lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/CommonTermsQuery.java Mon Dec  1 17:25:39 2014
@@ -85,7 +85,7 @@ public class CommonTermsQuery extends Qu
    * @param lowFreqOccur
    *          {@link Occur} used for low frequency terms
    * @param maxTermFrequency
-   *          a value in [0..1) (or absolute number >=1) representing the
+   *          a value in [0..1) (or absolute number &gt;=1) representing the
    *          maximum threshold of a terms document frequency to be considered a
    *          low frequency term.
    * @throws IllegalArgumentException
@@ -105,7 +105,7 @@ public class CommonTermsQuery extends Qu
    * @param lowFreqOccur
    *          {@link Occur} used for low frequency terms
    * @param maxTermFrequency
-   *          a value in [0..1) (or absolute number >=1) representing the
+   *          a value in [0..1) (or absolute number &gt;=1) representing the
    *          maximum threshold of a terms document frequency to be considered a
    *          low frequency term.
    * @param disableCoord

Modified: lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ReciprocalFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ReciprocalFloatFunction.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ReciprocalFloatFunction.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/ReciprocalFloatFunction.java Mon Dec  1 17:25:39 2014
@@ -31,7 +31,7 @@ import java.util.Map;
  * the float value of a field or function as exported by {@link org.apache.lucene.queries.function.ValueSource}.
  * <br>
  *
- * When a and b are equal, and x>=0, this function has a maximum value of 1 that drops as x increases.
+ * When a and b are equal, and x&gt;=0, this function has a maximum value of 1 that drops as x increases.
  * Increasing the value of a and b together results in a movement of the entire function to a flatter part of the curve.
  * <p>These properties make this an idea function for boosting more recent documents.
  * <p>Example:<code>  recip(ms(NOW,mydatefield),3.16e-11,1,1)</code>

Modified: lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queries/src/java/org/apache/lucene/queries/mlt/MoreLikeThis.java Mon Dec  1 17:25:39 2014
@@ -1,3 +1,5 @@
+package org.apache.lucene.queries.mlt;
+
 /**
  * Copyright 2004-2005 The Apache Software Foundation.
  *
@@ -13,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.lucene.queries.mlt;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -45,7 +46,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
-
 /**
  * Generate "more like this" similarity queries.
  * Based on this mail:
@@ -644,7 +644,7 @@ public final class MoreLikeThis {
   }
 
   /**
-   * Create a PriorityQueue from a word->tf map.
+   * Create a PriorityQueue from a word-&gt;tf map.
    *
    * @param words a map of words keyed on the word(String) with Int objects as the values.
    */

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/MultiFieldQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/MultiFieldQueryParser.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/MultiFieldQueryParser.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/MultiFieldQueryParser.java Mon Dec  1 17:25:39 2014
@@ -55,7 +55,7 @@ public class MultiFieldQueryParser exten
    * +(title:term1 body:term1) +(title:term2 body:term2)
    * </code>
    * 
-   * <p>When you pass a boost (title=>5 body=>10) you can get </p>
+   * <p>When you pass a boost (title=&gt;5 body=&gt;10) you can get </p>
    * 
    * <code>
    * +(title:term1^5.0 body:term1^10.0) +(title:term2^5.0 body:term2^10.0)

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/config/AbstractQueryConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/config/AbstractQueryConfig.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/config/AbstractQueryConfig.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/config/AbstractQueryConfig.java Mon Dec  1 17:25:39 2014
@@ -25,7 +25,7 @@ import java.util.HashMap;
  * It has operations to set, unset and get configuration values.
  * </p>
  * <p>
- * Each configuration is is a key->value pair. The key should be an unique
+ * Each configuration is is a key-&gt;value pair. The key should be an unique
  * {@link ConfigurationKey} instance and it also holds the value's type.
  * </p>
  * 

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java Mon Dec  1 17:25:39 2014
@@ -19,7 +19,6 @@ package org.apache.lucene.queryparser.fl
 
 import org.apache.lucene.queryparser.flexible.precedence.PrecedenceQueryParser;
 import org.apache.lucene.queryparser.flexible.standard.processors.BooleanQuery2ModifierNodeProcessor;
-import org.apache.lucene.queryparser.flexible.standard.processors.GroupQueryNodeProcessor;
 import org.apache.lucene.queryparser.flexible.standard.processors.StandardQueryNodeProcessorPipeline;
 import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
 
@@ -29,7 +28,7 @@ import org.apache.lucene.queryparser.fle
  * boolean precedence on it.
  * </p>
  * <p>
- * EXPERT: the precedence is enabled by removing {@link GroupQueryNodeProcessor} from the
+ * EXPERT: the precedence is enabled by removing {@link BooleanQuery2ModifierNodeProcessor} from the
  * {@link StandardQueryNodeProcessorPipeline} and appending {@link BooleanModifiersQueryNodeProcessor}
  * to the pipeline.
  * </p>

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java Mon Dec  1 17:25:39 2014
@@ -369,17 +369,6 @@ public class StandardQueryParser extends
   /**
    * Sets the default slop for phrases. If zero, then exact phrase matches are
    * required. Default value is zero.
-   * 
-   * @deprecated renamed to {@link #setPhraseSlop(int)}
-   */
-  @Deprecated
-  public void setDefaultPhraseSlop(int defaultPhraseSlop) {
-    getQueryConfigHandler().set(ConfigurationKeys.PHRASE_SLOP, defaultPhraseSlop);
-  }
-  
-  /**
-   * Sets the default slop for phrases. If zero, then exact phrase matches are
-   * required. Default value is zero.
    */
   @Override
   public void setPhraseSlop(int defaultPhraseSlop) {
@@ -510,18 +499,6 @@ public class StandardQueryParser extends
   public DateTools.Resolution getDateResolution() {
     return getQueryConfigHandler().get(ConfigurationKeys.DATE_RESOLUTION);
   }
-
-  /**
-   * Sets the {@link Resolution} used for each field
-   * 
-   * @param dateRes a collection that maps a field to its {@link Resolution}
-   * 
-   * @deprecated this method was renamed to {@link #setDateResolutionMap(Map)} 
-   */
-  @Deprecated
-  public void setDateResolution(Map<CharSequence, DateTools.Resolution> dateRes) {
-    setDateResolutionMap(dateRes);
-  }
   
   /**
    * Returns the field to {@link Resolution} map used to normalize each date field.

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldBoostMapFCListener.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldBoostMapFCListener.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldBoostMapFCListener.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldBoostMapFCListener.java Mon Dec  1 17:25:39 2014
@@ -27,7 +27,7 @@ import org.apache.lucene.queryparser.fle
 /**
  * This listener listens for every field configuration request and assign a
  * {@link ConfigurationKeys#BOOST} to the
- * equivalent {@link FieldConfig} based on a defined map: fieldName -> boostValue stored in
+ * equivalent {@link FieldConfig} based on a defined map: fieldName -&gt; boostValue stored in
  * {@link ConfigurationKeys#FIELD_BOOST_MAP}.
  * 
  * @see ConfigurationKeys#FIELD_BOOST_MAP

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldDateResolutionFCListener.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldDateResolutionFCListener.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldDateResolutionFCListener.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/FieldDateResolutionFCListener.java Mon Dec  1 17:25:39 2014
@@ -29,7 +29,7 @@ import org.apache.lucene.queryparser.fle
 /**
  * This listener listens for every field configuration request and assign a
  * {@link ConfigurationKeys#DATE_RESOLUTION} to the equivalent {@link FieldConfig} based
- * on a defined map: fieldName -> {@link Resolution} stored in
+ * on a defined map: fieldName -&gt; {@link Resolution} stored in
  * {@link ConfigurationKeys#FIELD_DATE_RESOLUTION_MAP}.
  * 
  * @see ConfigurationKeys#DATE_RESOLUTION

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/BooleanModifierNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/BooleanModifierNode.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/BooleanModifierNode.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/BooleanModifierNode.java Mon Dec  1 17:25:39 2014
@@ -19,12 +19,12 @@ package org.apache.lucene.queryparser.fl
 
 import org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
-import org.apache.lucene.queryparser.flexible.standard.processors.GroupQueryNodeProcessor;
+import org.apache.lucene.queryparser.flexible.standard.processors.BooleanQuery2ModifierNodeProcessor;
 
 /**
  * A {@link BooleanModifierNode} has the same behaviour as
  * {@link ModifierQueryNode}, it only indicates that this modifier was added by
- * {@link GroupQueryNodeProcessor} and not by the user. <br/>
+ * {@link BooleanQuery2ModifierNodeProcessor} and not by the user. <br/>
  * 
  * @see ModifierQueryNode
  */

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/simple/SimpleQueryParser.java Mon Dec  1 17:25:39 2014
@@ -43,7 +43,8 @@ import java.util.Map;
  * <p>
  * Any errors in query syntax will be ignored and the parser will attempt
  * to decipher what it can; however, this may mean odd or unexpected results.
- * <h4>Query Operators</h4>
+ * <p>
+ * <b>Query Operators</b>
  * <ul>
  *  <li>'{@code +}' specifies {@code AND} operation: <tt>token1+token2</tt>
  *  <li>'{@code |}' specifies {@code OR} operation: <tt>token1|token2</tt>
@@ -63,7 +64,7 @@ import java.util.Map;
  * For example, the following will evaluate {@code token1 OR token2} first,
  * then {@code AND} with {@code token3}:
  * <blockquote>token1 | token2 + token3</blockquote>
- * <h4>Escaping</h4>
+ * <b>Escaping</b>
  * <p>
  * An individual term may contain any possible character with certain characters
  * requiring escaping using a '{@code \}'.  The following characters will need to be escaped in

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeFilterBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeFilterBuilder.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeFilterBuilder.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeFilterBuilder.java Mon Dec  1 17:25:39 2014
@@ -35,7 +35,7 @@ import java.io.IOException;
  * attributes and the defaults if optional attributes are omitted. For more
  * detail on what each of the attributes actually do, consult the documentation
  * for {@link NumericRangeFilter}:
- * <table>
+ * <table summary="supported attributes">
  * <tr>
  * <th>Attribute name</th>
  * <th>Values</th>

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeQueryBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeQueryBuilder.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeQueryBuilder.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/NumericRangeQueryBuilder.java Mon Dec  1 17:25:39 2014
@@ -30,7 +30,7 @@ import org.w3c.dom.Element;
  * attributes and the defaults if optional attributes are omitted. For more
  * detail on what each of the attributes actually do, consult the documentation
  * for {@link NumericRangeQuery}:
- * <table>
+ * <table summary="supported attributes">
  * <tr>
  * <th>Attribute name</th>
  * <th>Values</th>

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/precedence/TestPrecedenceQueryParser.java Mon Dec  1 17:25:39 2014
@@ -434,14 +434,14 @@ public class TestPrecedenceQueryParser e
     Map<CharSequence, DateTools.Resolution> fieldMap = new HashMap<>();
     // set a field specific date resolution
     fieldMap.put(monthField, DateTools.Resolution.MONTH);
-    qp.setDateResolution(fieldMap);
+    qp.setDateResolutionMap(fieldMap);
 
     // set default date resolution to MILLISECOND
     qp.setDateResolution(DateTools.Resolution.MILLISECOND);
 
     // set second field specific date resolution
     fieldMap.put(hourField, DateTools.Resolution.HOUR);
-    qp.setDateResolution(fieldMap);
+    qp.setDateResolutionMap(fieldMap);
 
     // for this field no field specific date resolution has been set,
     // so verify if the default resolution is used

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java Mon Dec  1 17:25:39 2014
@@ -93,12 +93,12 @@ public class TestMultiAnalyzerQPHelper e
         .toString());
 
     // phrase after changing default slop
-    qp.setDefaultPhraseSlop(99);
+    qp.setPhraseSlop(99);
     assertEquals("\"(multi multi2) foo\"~99 bar", qp.parse("\"multi foo\" bar",
         "").toString());
     assertEquals("\"(multi multi2) foo\"~99 \"foo bar\"~2", qp.parse(
         "\"multi foo\" \"foo bar\"~2", "").toString());
-    qp.setDefaultPhraseSlop(0);
+    qp.setPhraseSlop(0);
 
     // non-default operator:
     qp.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);

Modified: lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java (original)
+++ lucene/dev/branches/lucene2878/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java Mon Dec  1 17:25:39 2014
@@ -744,14 +744,14 @@ public class TestQPHelper extends Lucene
     
     // set a field specific date resolution    
     dateRes.put(monthField, DateTools.Resolution.MONTH);
-    qp.setDateResolution(dateRes);
+    qp.setDateResolutionMap(dateRes);
 
     // set default date resolution to MILLISECOND
     qp.setDateResolution(DateTools.Resolution.MILLISECOND);
 
     // set second field specific date resolution
     dateRes.put(hourField, DateTools.Resolution.HOUR);
-    qp.setDateResolution(dateRes);
+    qp.setDateResolutionMap(dateRes);
 
     // for this field no field specific date resolution has been set,
     // so verify if the default resolution is used

Modified: lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsFormat.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsFormat.java Mon Dec  1 17:25:39 2014
@@ -36,7 +36,7 @@ import org.apache.lucene.util.IOUtils;
  *  created by {@link #longToBytes} during indexing.  At search time,
  *  the TermsEnum implementation {@link IDVersionSegmentTermsEnum}
  *  enables fast (using only the terms index when possible) lookup for
- *  whether a given ID was previously indexed with version > N (see
+ *  whether a given ID was previously indexed with version &gt; N (see
  *  {@link IDVersionSegmentTermsEnum#seekExact(BytesRef,long)}.
  *
  *  <p>This is most effective if the app assigns monotonically
@@ -58,12 +58,12 @@ import org.apache.lucene.util.IOUtils;
 
 public class IDVersionPostingsFormat extends PostingsFormat {
 
-  /** version must be >= this. */
+  /** version must be &gt;= this. */
   public static final long MIN_VERSION = 0;
 
   // TODO: we could delta encode instead, and keep the last bit:
 
-  /** version must be <= this, because we encode with ZigZag. */
+  /** version must be &lt;= this, because we encode with ZigZag. */
   public static final long MAX_VERSION = 0x3fffffffffffffffL;
 
   private final int minTermsInBlock;

Modified: lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java Mon Dec  1 17:25:39 2014
@@ -85,7 +85,7 @@ import org.apache.lucene.util.packed.Pac
 
 /**
  * This is just like {@link BlockTreeTermsWriter}, except it also stores a version per term, and adds a method to its TermsEnum
- * implementation to seekExact only if the version is >= the specified version.  The version is added to the terms index to avoid seeking if
+ * implementation to seekExact only if the version is &gt;= the specified version.  The version is added to the terms index to avoid seeking if
  * no term in the block has a high enough version.  The term blocks file is .tiv and the terms index extension is .tipv.
  *
  * @lucene.experimental

Modified: lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java (original)
+++ lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyQuery.java Mon Dec  1 17:25:39 2014
@@ -65,7 +65,7 @@ public class SlowFuzzyQuery extends Mult
    *  as the query term is considered similar to the query term if the edit distance
    *  between both terms is less than <code>length(term)*0.5</code>
    *  <p>
-   *  Alternatively, if <code>minimumSimilarity</code> is >= 1f, it is interpreted 
+   *  Alternatively, if <code>minimumSimilarity</code> is &gt;= 1f, it is interpreted 
    *  as a pure Levenshtein edit distance. For example, a value of <code>2f</code>
    *  will match all terms within an edit distance of <code>2</code> from the 
    *  query term. Edit distances specified in this way may not be fractional.

Modified: lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java (original)
+++ lucene/dev/branches/lucene2878/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/SlowFuzzyTermsEnum.java Mon Dec  1 17:25:39 2014
@@ -110,7 +110,7 @@ public final class SlowFuzzyTermsEnum ex
      * <p>The termCompare method in FuzzyTermEnum uses Levenshtein distance to 
      * calculate the distance between the given term and the comparing term. 
      * </p>
-     * <p>If the minSimilarity is >= 1.0, this uses the maxEdits as the comparison.
+     * <p>If the minSimilarity is &gt;= 1.0, this uses the maxEdits as the comparison.
      * Otherwise, this method uses the following logic to calculate similarity.
      * <pre>
      *   similarity = 1 - ((float)distance / (float) (prefixLength + Math.min(textlen, targetlen)));

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java Mon Dec  1 17:25:39 2014
@@ -48,7 +48,7 @@ import org.apache.lucene.spatial.query.S
  * be coerced into compatibility.
  * <p/>
  * Note that a SpatialStrategy is not involved with the Lucene stored field
- * values of shapes, which is immaterial to indexing & search.
+ * values of shapes, which is immaterial to indexing and search.
  * <p/>
  * Thread-safe.
  *

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxSimilarityValueSource.java Mon Dec  1 17:25:39 2014
@@ -33,7 +33,7 @@ import java.util.Map;
  * {@link ValueSource} in which {@link FunctionValues#objectVal(int)} returns a {@link
  * com.spatial4j.core.shape.Rectangle}.
  * <p/>
- * Implementers: remember to implement equals & hashCode if you have
+ * Implementers: remember to implement equals and hashCode if you have
  * fields!
  *
  * @lucene.experimental

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java Mon Dec  1 17:25:39 2014
@@ -51,8 +51,9 @@ import com.spatial4j.core.shape.Shape;
  * coordinates in numeric fields. It supports all {@link SpatialOperation}s and
  * has a custom overlap relevancy. It is based on GeoPortal's <a
  * href="http://geoportal.svn.sourceforge.net/svnroot/geoportal/Geoportal/trunk/src/com/esri/gpt/catalog/lucene/SpatialClauseAdapter.java">SpatialClauseAdapter</a>.
- *
- * <h4>Characteristics:</h4>
+ * <p>
+ * <b>Characteristics:</b>
+ * <p>
  * <ul>
  * <li>Only indexes Rectangles; just one per field value. Other shapes can be provided
  * and the bounding box will be used.</li>
@@ -60,8 +61,9 @@ import com.spatial4j.core.shape.Shape;
  * <li>Supports most {@link SpatialOperation}s but not Overlaps.</li>
  * <li>Uses the DocValues API for any sorting / relevancy.</li>
  * </ul>
- *
- * <h4>Implementation:</h4>
+ * <p>
+ * <b>Implementation:</b>
+ * <p>
  * This uses 4 double fields for minX, maxX, minY, maxY
  * and a boolean to mark a dateline cross. Depending on the particular {@link
  * SpatialOperation}s, there are a variety of {@link NumericRangeQuery}s to be

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeFilter.java Mon Dec  1 17:25:39 2014
@@ -31,7 +31,7 @@ import java.io.IOException;
 import java.util.Iterator;
 
 /**
- * Traverses a {@link SpatialPrefixTree} indexed field, using the template &
+ * Traverses a {@link SpatialPrefixTree} indexed field, using the template and
  * visitor design patterns for subclasses to guide the traversal and collect
  * matching documents.
  * <p/>
@@ -257,7 +257,7 @@ public abstract class AbstractVisitingPr
     }
 
     /**
-     * Called when doing a divide & conquer to find the next intersecting cells
+     * Called when doing a divide and conquer to find the next intersecting cells
      * of the query shape that are beneath {@code cell}. {@code cell} is
      * guaranteed to have an intersection and thus this must return some number
      * of nodes.

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Mon Dec  1 17:25:39 2014
@@ -40,8 +40,9 @@ import com.spatial4j.core.shape.Shape;
  * subclasses are {@link RecursivePrefixTreeStrategy} and {@link
  * TermQueryPrefixTreeStrategy}.  This strategy is most effective as a fast
  * approximate spatial search filter.
- *
- * <h4>Characteristics:</h4>
+ * <p>
+ * <b>Characteristics:</b>
+ * <p>
  * <ul>
  * <li>Can index any shape; however only {@link RecursivePrefixTreeStrategy}
  * can effectively search non-point shapes.</li>
@@ -64,8 +65,9 @@ import com.spatial4j.core.shape.Shape;
  * it doesn't scale to large numbers of points nor is it real-time-search
  * friendly.</li>
  * </ul>
- *
- * <h4>Implementation:</h4>
+ * <p>
+ * <b>Implementation:</b>
+ * <p>
  * The {@link SpatialPrefixTree} does most of the work, for example returning
  * a list of terms representing grids of various sizes for a supplied shape.
  * An important

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/CellIterator.java Mon Dec  1 17:25:39 2014
@@ -43,7 +43,7 @@ public abstract class CellIterator imple
   //public SpatialRelation getShapeRel()
 
   /**
-   * Gets the next cell that is >= {@code fromCell}, compared using non-leaf bytes. If it returns null then
+   * Gets the next cell that is &gt;= {@code fromCell}, compared using non-leaf bytes. If it returns null then
    * the iterator is exhausted.
    */
   public Cell nextFrom(Cell fromCell) {

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/GeohashPrefixTree.java Mon Dec  1 17:25:39 2014
@@ -65,7 +65,7 @@ public class GeohashPrefixTree extends L
       throw new IllegalArgumentException("maxLen must be [1-"+MAXP+"] but got "+ maxLevels);
   }
 
-  /** Any more than this and there's no point (double lat & lon are the same). */
+  /** Any more than this and there's no point (double lat and lon are the same). */
   public static int getMaxLevelsPossible() {
     return GeohashUtils.MAX_PRECISION;
   }

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyCell.java Mon Dec  1 17:25:39 2014
@@ -25,7 +25,7 @@ import org.apache.lucene.util.StringHelp
 
 import java.util.Collection;
 
-/** The base for the original two SPT's: Geohash & Quad. Don't subclass this for new SPTs.
+/** The base for the original two SPT's: Geohash and Quad. Don't subclass this for new SPTs.
  * @lucene.internal */
 //public for RPT pruneLeafyBranches code
 public abstract class LegacyCell implements Cell {
@@ -169,7 +169,7 @@ public abstract class LegacyCell impleme
   protected abstract Collection<Cell> getSubCells();
 
   /**
-   * {@link #getSubCells()}.size() -- usually a constant. Should be >=2
+   * {@link #getSubCells()}.size() -- usually a constant. Should be &gt;=2
    */
   public abstract int getSubCellsSize();
 

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/LegacyPrefixTree.java Mon Dec  1 17:25:39 2014
@@ -25,7 +25,7 @@ import org.apache.lucene.util.BytesRef;
 
 import java.util.Arrays;
 
-/** The base for the original two SPT's: Geohash & Quad. Don't subclass this for new SPTs.
+/** The base for the original two SPT's: Geohash and Quad. Don't subclass this for new SPTs.
  * @lucene.internal */
 abstract class LegacyPrefixTree extends SpatialPrefixTree {
   public LegacyPrefixTree(SpatialContext ctx, int maxLevels) {

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTree.java Mon Dec  1 17:25:39 2014
@@ -64,7 +64,7 @@ public abstract class SpatialPrefixTree 
    * grid, such that you can get a grid with just the right amount of
    * precision.
    *
-   * @param dist >= 0
+   * @param dist {@code >= 0}
    * @return level [1 to maxLevels]
    */
   public abstract int getLevelForDistance(double dist);
@@ -75,7 +75,7 @@ public abstract class SpatialPrefixTree 
    * may over-estimate.
    *
    * @param level [1 to maxLevels]
-   * @return > 0
+   * @return {@code > 0}
    */
   public abstract double getDistanceForLevel(int level);
 

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/query/SpatialArgs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/query/SpatialArgs.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/query/SpatialArgs.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/query/SpatialArgs.java Mon Dec  1 17:25:39 2014
@@ -76,7 +76,7 @@ public class SpatialArgs {
    * looks at {@link #getDistErr()}, {@link #getDistErrPct()}, and {@code
    * defaultDistErrPct}.
    * @param defaultDistErrPct 0 to 0.5
-   * @return >= 0
+   * @return {@code >= 0}
    */
   public double resolveDistErr(SpatialContext ctx, double defaultDistErrPct) {
     if (distErr != null)
@@ -137,7 +137,7 @@ public class SpatialArgs {
    * The acceptable error of the shape.  This effectively inflates the
    * size of the shape but should not shrink it.
    *
-   * @return >= 0
+   * @return {@code >= 0}
    */
   public Double getDistErr() {
     return distErr;

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java Mon Dec  1 17:25:39 2014
@@ -47,7 +47,9 @@ import org.apache.lucene.spatial.util.Va
  * Simple {@link SpatialStrategy} which represents Points in two numeric {@link
  * DoubleField}s.  The Strategy's best feature is decent distance sort.
  *
- * <h4>Characteristics:</h4>
+ * <p>
+ * <b>Characteristics:</b>
+ * <p>
  * <ul>
  * <li>Only indexes points; just one per field value.</li>
  * <li>Can query by a rectangle or circle.</li>
@@ -59,9 +61,11 @@ import org.apache.lucene.spatial.util.Va
  * searching with a Circle.</li>
  * </ul>
  *
- * <h4>Implementation:</h4>
+ * <p>
+ * <b>Implementation:</b>
+ * <p>
  * This is a simple Strategy.  Search works with {@link NumericRangeQuery}s on
- * an x & y pair of fields.  A Circle query does the same bbox query but adds a
+ * an x and y pair of fields.  A Circle query does the same bbox query but adds a
  * ValueSource filter on
  * {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}.
  * <p />

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/JtsPolygonTest.java Mon Dec  1 17:25:39 2014
@@ -82,7 +82,7 @@ public class JtsPolygonTest extends Stra
 
   /**
    * A PrefixTree pruning optimization gone bad.
-   * See <a href="https://issues.apache.org/jira/browse/LUCENE-4770>LUCENE-4770</a>.
+   * See <a href="https://issues.apache.org/jira/browse/LUCENE-4770">LUCENE-4770</a>.
    */
   @Test
   public void testBadPrefixTreePrune() throws Exception {

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/RandomSpatialOpFuzzyPrefixTreeTest.java Mon Dec  1 17:25:39 2014
@@ -407,7 +407,7 @@ public class RandomSpatialOpFuzzyPrefixT
 
   /**
    * An aggregate of 2 shapes. Unfortunately we can't simply use a ShapeCollection because:
-   * (a) ambiguity between CONTAINS & WITHIN for equal shapes, and
+   * (a) ambiguity between CONTAINS and WITHIN for equal shapes, and
    * (b) adjacent pairs could as a whole contain the input shape.
    * The tests here are sensitive to these matters, although in practice ShapeCollection
    * is fine.

Modified: lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeTest.java Mon Dec  1 17:25:39 2014
@@ -78,7 +78,7 @@ public class SpatialPrefixTreeTest exten
   }
   /**
    * A PrefixTree pruning optimization gone bad, applicable when optimize=true.
-   * See <a href="https://issues.apache.org/jira/browse/LUCENE-4770>LUCENE-4770</a>.
+   * See <a href="https://issues.apache.org/jira/browse/LUCENE-4770">LUCENE-4770</a>.
    */
   @Test
   public void testBadPrefixTreePrune() throws Exception {

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/spell/DirectSpellChecker.java Mon Dec  1 17:25:39 2014
@@ -73,12 +73,12 @@ public class DirectSpellChecker {
   private int maxInspections = 5;
   /** minimum accuracy for a term to match */
   private float accuracy = SpellChecker.DEFAULT_ACCURACY;
-  /** value in [0..1] (or absolute number >=1) representing the minimum
+  /** value in [0..1] (or absolute number &gt;= 1) representing the minimum
     * number of documents (of the total) where a term should appear. */
   private float thresholdFrequency = 0f;
   /** minimum length of a query word to return suggestions */
   private int minQueryLength = 4;
-  /** value in [0..1] (or absolute number >=1) representing the maximum
+  /** value in [0..1] (or absolute number &gt;= 1) representing the maximum
    *  number of documents (of the total) a query term can appear in to
    *  be corrected. */
   private float maxQueryFrequency = 0.01f;

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java Mon Dec  1 17:25:39 2014
@@ -82,7 +82,6 @@ import org.apache.lucene.util.Accountabl
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
-import org.apache.lucene.util.Version;
 
 // TODO:
 //   - a PostingsFormat that stores super-high-freq terms as
@@ -125,7 +124,6 @@ public class AnalyzingInfixSuggester ext
   protected final Analyzer queryAnalyzer;
   /** Analyzer used at index time */
   protected final Analyzer indexAnalyzer;
-  final Version matchVersion;
   private final Directory dir;
   final int minPrefixChars;
   private final boolean commitOnBuild;
@@ -152,15 +150,6 @@ public class AnalyzingInfixSuggester ext
     this(dir, analyzer, analyzer, DEFAULT_MIN_PREFIX_CHARS, false);
   }
 
-  /**
-   * @deprecated Use {@link #AnalyzingInfixSuggester(Directory, Analyzer)}
-   */
-  @Deprecated
-  public AnalyzingInfixSuggester(Version matchVersion, Directory dir, Analyzer analyzer) throws IOException {
-    this(matchVersion, dir, analyzer, analyzer, DEFAULT_MIN_PREFIX_CHARS, false);
-  }
-
-
   /** Create a new instance, loading from a previously built
    *  AnalyzingInfixSuggester directory, if it exists.  This directory must be
    *  private to the infix suggester (i.e., not an external
@@ -178,15 +167,6 @@ public class AnalyzingInfixSuggester ext
    */
   public AnalyzingInfixSuggester(Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int minPrefixChars,
                                  boolean commitOnBuild) throws IOException {
-     this(indexAnalyzer.getVersion(), dir, indexAnalyzer, queryAnalyzer, minPrefixChars, commitOnBuild);
-  }
-
-  /**
-   * @deprecated Use {@link #AnalyzingInfixSuggester(Directory, Analyzer, Analyzer, int, boolean)}
-   */
-  @Deprecated
-  public AnalyzingInfixSuggester(Version matchVersion, Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer, int minPrefixChars,
-                                 boolean commitOnBuild) throws IOException {
 
     if (minPrefixChars < 0) {
       throw new IllegalArgumentException("minPrefixChars must be >= 0; got: " + minPrefixChars);
@@ -194,7 +174,6 @@ public class AnalyzingInfixSuggester ext
 
     this.queryAnalyzer = queryAnalyzer;
     this.indexAnalyzer = indexAnalyzer;
-    this.matchVersion = matchVersion;
     this.dir = dir;
     this.minPrefixChars = minPrefixChars;
     this.commitOnBuild = commitOnBuild;

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Mon Dec  1 17:25:39 2014
@@ -119,7 +119,7 @@ import static org.apache.lucene.util.aut
 public class AnalyzingSuggester extends Lookup {
  
   /**
-   * FST<Weight,Surface>: 
+   * FST&lt;Weight,Surface&gt;: 
    *  input is the analyzed form, with a null byte between terms
    *  weights are encoded as costs: (Integer.MAX_VALUE-weight)
    *  surface is the original, unanalyzed form.
@@ -912,12 +912,12 @@ public class AnalyzingSuggester extends 
     throw new UnsupportedOperationException();
   }
   
-  /** cost -> weight */
+  /** cost -&gt; weight */
   private static int decodeWeight(long encoded) {
     return (int)(Integer.MAX_VALUE - encoded);
   }
   
-  /** weight -> cost */
+  /** weight -&gt; cost */
   private static int encodeWeight(long value) {
     if (value < 0 || value > Integer.MAX_VALUE) {
       throw new UnsupportedOperationException("cannot encode value: " + value);

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java Mon Dec  1 17:25:39 2014
@@ -41,7 +41,6 @@ import org.apache.lucene.search.TopField
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Version;
 
 // TODO:
 // - allow to use the search score
@@ -97,15 +96,7 @@ public class BlendedInfixSuggester exten
    * directory, if it exists.
    */
   public BlendedInfixSuggester(Directory dir, Analyzer analyzer) throws IOException {
-    this(analyzer.getVersion(), dir, analyzer);
-  }
-
-  /**
-   * @deprecated Use {@link #BlendedInfixSuggester(Directory, Analyzer)}
-   */
-  @Deprecated
-  public BlendedInfixSuggester(Version matchVersion, Directory dir, Analyzer analyzer) throws IOException {
-    super(matchVersion, dir, analyzer);
+    super(dir, analyzer);
     this.blenderType = BlenderType.POSITION_LINEAR;
     this.numFactor = DEFAULT_NUM_FACTOR;
   }
@@ -122,16 +113,7 @@ public class BlendedInfixSuggester exten
    */
   public BlendedInfixSuggester(Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer,
                                int minPrefixChars, BlenderType blenderType, int numFactor, boolean commitOnBuild) throws IOException {
-    this(indexAnalyzer.getVersion(), dir, indexAnalyzer, queryAnalyzer, minPrefixChars, blenderType, numFactor, commitOnBuild);
-  }
-
-  /**
-   * @deprecated Use {@link #BlendedInfixSuggester(Directory, Analyzer, Analyzer, int, BlendedInfixSuggester.BlenderType, int, boolean)}
-   */
-  @Deprecated
-  public BlendedInfixSuggester(Version matchVersion, Directory dir, Analyzer indexAnalyzer, Analyzer queryAnalyzer,
-                               int minPrefixChars, BlenderType blenderType, int numFactor, boolean commitOnBuild) throws IOException {
-    super(matchVersion, dir, indexAnalyzer, queryAnalyzer, minPrefixChars, commitOnBuild);
+    super(dir, indexAnalyzer, queryAnalyzer, minPrefixChars, commitOnBuild);
     this.blenderType = blenderType;
     this.numFactor = numFactor;
   }

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java Mon Dec  1 17:25:39 2014
@@ -734,12 +734,12 @@ public class FreeTextSuggester extends L
     }
   }
 
-  /** weight -> cost */
+  /** weight -&gt; cost */
   private long encodeWeight(long ngramCount) {
     return Long.MAX_VALUE - ngramCount;
   }
 
-  /** cost -> weight */
+  /** cost -&gt; weight */
   //private long decodeWeight(Pair<Long,BytesRef> output) {
   private long decodeWeight(Long output) {
     assert output != null;

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Mon Dec  1 17:25:39 2014
@@ -118,7 +118,7 @@ public final class FuzzySuggester extend
   }
   
   /**
-   * Creates a {@link FuzzySuggester} instance with an index & a query analyzer initialized with default values.
+   * Creates a {@link FuzzySuggester} instance with an index and query analyzer initialized with default values.
    * 
    * @param indexAnalyzer
    *           Analyzer that will be used for analyzing suggestions while building the index.
@@ -146,7 +146,7 @@ public final class FuzzySuggester extend
    *        to expand from the analyzed form.  Set this to -1 for
    *        no limit.
    * @param preservePositionIncrements Whether position holes should appear in the automaton
-   * @param maxEdits must be >= 0 and <= {@link LevenshteinAutomata#MAXIMUM_SUPPORTED_DISTANCE} .
+   * @param maxEdits must be &gt;= 0 and &lt;= {@link LevenshteinAutomata#MAXIMUM_SUPPORTED_DISTANCE} .
    * @param transpositions <code>true</code> if transpositions should be treated as a primitive 
    *        edit operation. If this is false, comparisons will implement the classic
    *        Levenshtein algorithm.

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java Mon Dec  1 17:25:39 2014
@@ -243,7 +243,7 @@ public class FSTCompletionLookup extends
     }
   }
   
-  /** weight -> cost */
+  /** weight -&gt; cost */
   private static int encodeWeight(long value) {
     if (value < Integer.MIN_VALUE || value > Integer.MAX_VALUE) {
       throw new UnsupportedOperationException("cannot encode value: " + value);

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Mon Dec  1 17:25:39 2014
@@ -247,12 +247,12 @@ public class WFSTCompletionLookup extend
     }
   }
   
-  /** cost -> weight */
+  /** cost -&gt; weight */
   private static int decodeWeight(long encoded) {
     return (int)(Integer.MAX_VALUE - encoded);
   }
   
-  /** weight -> cost */
+  /** weight -&gt; cost */
   private static int encodeWeight(long value) {
     if (value < 0 || value > Integer.MAX_VALUE) {
       throw new UnsupportedOperationException("cannot encode value: " + value);

Modified: lucene/dev/branches/lucene2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java (original)
+++ lucene/dev/branches/lucene2878/lucene/suggest/src/test/org/apache/lucene/search/suggest/DocumentDictionaryTest.java Mon Dec  1 17:25:39 2014
@@ -53,7 +53,7 @@ public class DocumentDictionaryTest exte
   static final String PAYLOAD_FIELD_NAME = "p1";
   static final String CONTEXT_FIELD_NAME = "c1";
   
-  /** Returns Pair(list of invalid document terms, Map of document term -> document) */
+  /** Returns Pair(list of invalid document terms, Map of document term -&gt; document) */
   private Map.Entry<List<String>, Map<String, Document>> generateIndexDocuments(int ndocs, boolean requiresPayload, boolean requiresContexts) {
     Map<String, Document> docs = new HashMap<>();
     List<String> invalidDocTerms = new ArrayList<>();

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/analysis/MockGraphTokenFilter.java Mon Dec  1 17:25:39 2014
@@ -26,7 +26,7 @@ import org.apache.lucene.util.TestUtil;
 // TODO: sometimes remove tokens too...?
 
 /** Randomly inserts overlapped (posInc=0) tokens with
- *  posLength sometimes > 1.  The chain must have
+ *  posLength sometimes &gt; 1.  The chain must have
  *  an OffsetAttribute.  */
 
 public final class MockGraphTokenFilter extends LookaheadTokenFilter<LookaheadTokenFilter.Position> {

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java Mon Dec  1 17:25:39 2014
@@ -143,11 +143,6 @@ public class AssertingStoredFieldsFormat
     }
 
     @Override
-    public void abort() {
-      in.abort();
-    }
-
-    @Override
     public void finish(FieldInfos fis, int numDocs) throws IOException {
       assert docStatus == (numDocs > 0 ? Status.FINISHED : Status.UNDEFINED);
       in.finish(fis, numDocs);

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java Mon Dec  1 17:25:39 2014
@@ -195,11 +195,6 @@ public class AssertingTermVectorsFormat 
     }
 
     @Override
-    public void abort() {
-      in.abort();
-    }
-
-    @Override
     public void finish(FieldInfos fis, int numDocs) throws IOException {
       assert docCount == numDocs;
       assert docStatus == (numDocs > 0 ? Status.FINISHED : Status.UNDEFINED);

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java Mon Dec  1 17:25:39 2014
@@ -64,14 +64,6 @@ class CrankyStoredFieldsFormat extends S
     }
 
     @Override
-    public void abort() {
-      delegate.abort();
-      if (random.nextInt(100) == 0) {
-        throw new RuntimeException(new IOException("Fake IOException from StoredFieldsWriter.abort()"));
-      }
-    }
-
-    @Override
     public void finish(FieldInfos fis, int numDocs) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from StoredFieldsWriter.finish()");

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java Mon Dec  1 17:25:39 2014
@@ -64,14 +64,6 @@ class CrankyTermVectorsFormat extends Te
     }
     
     @Override
-    public void abort() {
-      delegate.abort();
-      if (random.nextInt(100) == 0) {
-        throw new RuntimeException(new IOException("Fake IOException from TermVectorsWriter.abort()"));
-      }
-    }
-    
-    @Override
     public int merge(MergeState mergeState) throws IOException {
       if (random.nextInt(100) == 0) {
         throw new IOException("Fake IOException from TermVectorsWriter.merge()");

Modified: lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java?rev=1642718&r1=1642717&r2=1642718&view=diff
==============================================================================
--- lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java (original)
+++ lucene/dev/branches/lucene2878/lucene/test-framework/src/java/org/apache/lucene/index/BaseNormsFormatTestCase.java Mon Dec  1 17:25:39 2014
@@ -347,7 +347,7 @@ public abstract class BaseNormsFormatTes
   
   // TODO: test thread safety (e.g. across different fields) explicitly here
 
-  /** 
+  /*
    * LUCENE-6006: Tests undead norms.
    *                                 .....            
    *                             C C  /