You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by iv...@apache.org on 2022/12/15 14:10:05 UTC

[lucene-solr] branch branch_8_11 updated: Fix flat polygons incorrectly containing intersecting geometries (#12022)

This is an automated email from the ASF dual-hosted git repository.

ivera pushed a commit to branch branch_8_11
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8_11 by this push:
     new c209a8738f4 Fix flat polygons incorrectly containing intersecting geometries (#12022)
c209a8738f4 is described below

commit c209a8738f4ecbef954102188a715f5e7e165666
Author: Craig Taverner <cr...@amanzi.com>
AuthorDate: Thu Dec 15 14:56:09 2022 +0100

    Fix flat polygons incorrectly containing intersecting geometries (#12022)
    
    # Conflicts:
    #       lucene/CHANGES.txt
    #       lucene/core/src/test/org/apache/lucene/document/TestLatLonShape.java
---
 lucene/CHANGES.txt                                 | 1978 ++++++++++----------
 .../org/apache/lucene/document/ShapeField.java     |    6 +
 .../apache/lucene/document/TestLatLonShape.java    |   36 +-
 3 files changed, 1028 insertions(+), 992 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index ed81059c415..511526d9c53 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -7,11 +7,11 @@ http://s.apache.org/luceneversions
 
 Bug Fixes
 ---------------------
-* LUCENE-9580: Fix bug in the polygon tessellator when introducing collinear edges during polygon 
-  splitting. (Ignacio Vera)  
-  
+* LUCENE-9580: Fix bug in the polygon tessellator when introducing collinear edges during polygon
+  splitting. (Ignacio Vera)
+
 * LUCENE-10470: Check if polygon has been successfully tessellated before we fail (we are failing some valid
-  tessellations) and allow filtering edges that fold on top of the previous one. (Ignacio Vera)    
+  tessellations) and allow filtering edges that fold on top of the previous one. (Ignacio Vera)
 
 * LUCENE-10563: Fix failure to tessellate complex polygon (Craig Taverner)
 
@@ -21,6 +21,8 @@ Bug Fixes
 * GITHUB#11986: Fix algorithm that chooses the bridge between a polygon and a hole when there is
   common vertex. (Ignacio Vera)
 
+* GITHUB#12020: Fixes bug whereby very flat polygons can incorrectly contain intersecting geometries. (Craig Taverner)
+
 ======================= Lucene 8.11.2 =======================
 
 Bug Fixes
@@ -257,16 +259,16 @@ New Features
 
 * LUCENE-9574 A token filter to drop tokens that match all specified flags. (Gus Heck, Uwe Schindler)
 
-* LUCENE-9537:  Added smoothingScore method and default implementation to 
-  Scorable abstract class.  The smoothing score allows scorers to calculate a 
-  score for a document where the search term or subquery is not present.  The 
-  smoothing score acts like an idf so that documents that do not have terms or 
-  subqueries that are more frequent in the index are not penalized as much as 
-  documents that do not have less frequent terms or subqueries and prevents 
-  scores which are the product or terms or subqueries from going to zero. Added 
-  the implementation of the Indri AND and the IndriDirichletSimilarity from the 
-  academic Indri search engine: http://www.lemurproject.org/indri.php. 
-  (Cameron VandenBerg)   
+* LUCENE-9537:  Added smoothingScore method and default implementation to
+  Scorable abstract class.  The smoothing score allows scorers to calculate a
+  score for a document where the search term or subquery is not present.  The
+  smoothing score acts like an idf so that documents that do not have terms or
+  subqueries that are more frequent in the index are not penalized as much as
+  documents that do not have less frequent terms or subqueries and prevents
+  scores which are the product or terms or subqueries from going to zero. Added
+  the implementation of the Indri AND and the IndriDirichletSimilarity from the
+  academic Indri search engine: http://www.lemurproject.org/indri.php.
+  (Cameron VandenBerg)
 
 * LUCENE-9694: New tool for creating a deterministic index to enable benchmarking changes
   on a consistent multi-segment index even when they require re-indexing. (Haoyu Zhai)
@@ -317,7 +319,7 @@ Bug Fixes
 
 * LUCENE-9958: Fixed performance regression for boolean queries that configure a
   minimum number of matching clauses. (Adrien Grand, Matt Weber)
-  
+
 * LUCENE-9953: LongValueFacetCounts should count each document at most once when determining
   the total count for a dimension. Prior to this fix, multi-value docs could contribute a > 1
   count to the dimension count. (Greg Miller)
@@ -350,7 +352,7 @@ Bug Fixes
 ---------------------
 
 * LUCENE-9870: Fix Circle2D intersectsLine t-value (distance) range clamp (Jørgen Nystad)
-  
+
 * LUCENE-9744: NPE on a degenerate query in MinimumShouldMatchIntervalsSource
   $MinimumMatchesIterator.getSubMatches(). (Alan Woodward)
 
@@ -373,9 +375,9 @@ API Changes
 New Features
 ---------------------
 
-* LUCENE-9552: New LatLonPoint query that accepts an array of LatLonGeometries. (Ignacio Vera)  
+* LUCENE-9552: New LatLonPoint query that accepts an array of LatLonGeometries. (Ignacio Vera)
 
-* LUCENE-9641: LatLonPoint query support for spatial relationships. (Ignacio Vera) 
+* LUCENE-9641: LatLonPoint query support for spatial relationships. (Ignacio Vera)
 
 * LUCENE-9553: New XYPoint query that accepts an array of XYGeometries. (Ignacio Vera)
 
@@ -404,7 +406,7 @@ Optimizations
 
 * LUCENE-9536: Reduced memory usage for OrdinalMap when a segment has all
   values. (Julie Tibshirani via Adrien Grand)
-  
+
 * LUCENE-9021: QueryParser: re-use the LookaheadSuccess exception. (Przemek Bruski via Mikhail Khludnev)
 
 * LUCENE-9636: Faster decoding of postings for some numbers of bits per value.
@@ -425,18 +427,18 @@ Bug Fixes
   of long tokens when discardCompoundToken is activated. (Jim Ferenczi)
 
 * LUCENE-9595: Make Component2D#withinPoint implementations consistent with ShapeQuery logic.
-  (Ignacio Vera)   
+  (Ignacio Vera)
 
-* LUCENE-9606: Wrap boolean queries generated by shape fields with a Constant score query. (Ignacio Vera)  
+* LUCENE-9606: Wrap boolean queries generated by shape fields with a Constant score query. (Ignacio Vera)
 
 * LUCENE-9635: BM25FQuery - Mask encoded norm long value in array lookup.
   (Yilun Cui)
-  
+
 * LUCENE-9617: Fix per-field memory leak in IndexWriter.deleteAll(). Reset next available internal
   field number to 0 on FieldInfos.clear(), to avoid wasting FieldInfo references. (Michael Froh)
 
-* LUCENE-9642: When encoding triangles in ShapeField, make sure generated triangles are CCW by rotating 
-  triangle points before checking triangle orientation. (Ignacio Vera)  
+* LUCENE-9642: When encoding triangles in ShapeField, make sure generated triangles are CCW by rotating
+  triangle points before checking triangle orientation. (Ignacio Vera)
 
 * LUCENE-9661: Fix deadlock in TermsEnum.EMPTY that occurs when trying to initialize TermsEnum and BaseTermsEnum
   at the same time (Namgyu Kim)
@@ -446,7 +448,7 @@ Other
 
 * SOLR-14995: Update Jetty to 9.4.34 (Mike Drob)
 
-* LUCENE-9637: Removes some unused code and replaces the Point implementation on ShapeField/ShapeQuery 
+* LUCENE-9637: Removes some unused code and replaces the Point implementation on ShapeField/ShapeQuery
   random tests. (Ignacio Vera)
 
 ======================= Lucene 8.7.0 =======================
@@ -466,7 +468,7 @@ API Changes
   were moved to their own package. TopFieldCollector sets
   TotalHits.relation to GREATER_THAN_OR_EQUAL_TO,
   as soon as the requested total hits threshold is reached, even though
-  in some cases no skipping optimization is applied and all hits are collected.  
+  in some cases no skipping optimization is applied and all hits are collected.
 
 New Features
 ---------------------
@@ -505,18 +507,18 @@ Improvements
   values, and sometimes exercise Weight#scorer instead of Weight#bulkScorer for top-level
   queries. (Julie Tibshirani)
 
-* LUCENE-9511: Include StoredFieldsWriter in DWPT accounting to ensure that it's 
+* LUCENE-9511: Include StoredFieldsWriter in DWPT accounting to ensure that it's
   heap consumption is taken into account when IndexWriter stalls or should flush
   DWPTs. (Simon Willnauer)
 
-* LUCENE-9514: Include TermVectorsWriter in DWPT accounting to ensure that it's 
+* LUCENE-9514: Include TermVectorsWriter in DWPT accounting to ensure that it's
   heap consumption is taken into account when IndexWriter stalls or should flush
   DWPTs. (Simon Willnauer)
 
 * LUCENE-9523: In query shapes over shape fields, skip points while traversing the
   BKD tree when the relationship with the document is already known. (Ignacio Vera)
 
-* LUCENE-9539: Use more compact datastructures to represent sorted doc-values in memory when 
+* LUCENE-9539: Use more compact datastructures to represent sorted doc-values in memory when
   sorting a segment before flush and in SortingCodecReader. (Simon Willnauer)
 
 * LUCENE-9458: WordDelimiterGraphFilter should order tokens at the same position by endOffset to
@@ -576,7 +578,7 @@ Changes in Runtime Behavior
 
 * LUCENE-9539: SortingCodecReader now doesn't cache doc values fields anymore. Previously, SortingCodecReader
   used to cache all doc values fields after they were loaded into memory. This reader should only be used
-  to sort segments after the fact using IndexWriter#addIndices. (Simon Willnauer) 
+  to sort segments after the fact using IndexWriter#addIndices. (Simon Willnauer)
 
 
 Other
@@ -634,7 +636,7 @@ API Changes
 * LUCENE-9340: Deprecate SimpleBindings#add(SortField). (Alan Woodward)
 
 * LUCENE-9345: MergeScheduler is now decoupled from IndexWriter. Instead it accepts a MergeSource
-  interface that offers the basic methods to acquire pending merges, run the merge and do accounting 
+  interface that offers the basic methods to acquire pending merges, run the merge and do accounting
   around it. (Simon Willnauer)
 
 * LUCENE-9349: QueryVisitor.consumeTermsMatching() now takes a
@@ -684,7 +686,7 @@ Improvements
 
 * LUCENE-9359: SegmentInfos#readCommit now always returns a
   CorruptIndexException if the content of the file is invalid. (Adrien Grand)
-  
+
 * LUCENE-9393: Make FunctionScoreQuery use ScoreMode.COMPLETE for creating the inner query weight when
   ScoreMode.TOP_DOCS is requested. (Tomás Fernández Löbbe)
 
@@ -1199,7 +1201,7 @@ Optimizations
 
 * LUCENE-8860: add additional leaf node level optimizations in LatLonShapeBoundingBoxQuery.
   (Igor Motov via Ignacio Vera)
-  
+
 * LUCENE-8968: Improve performance of WITHIN and DISJOINT queries for Shape queries by
   doing just one pass whenever possible. (Ignacio Vera)
 
@@ -1263,7 +1265,7 @@ New Features
  (Gert Morten Paimla via Tomoko Uchida)
 
 * LUCENE-8815: Provide a DoubleValues implementation for retrieving the value of features without
-  requiring a separate numeric field. Note that as feature values are stored with only 8 bits of 
+  requiring a separate numeric field. Note that as feature values are stored with only 8 bits of
   mantissa the values returned may have a delta from the original values indexed.
   (Colin Goodheart-Smithe via Adrien Grand)
 
@@ -1407,8 +1409,8 @@ API Changes
 * LUCENE-3041: A query introspection API has been added.  Queries should
   implement a visit() method, taking a QueryVisitor, and either pass the
   visitor down to any child queries, or call a visitX() or consumeX() method
-  on it.  All locations in the code that called Weight.extractTerms() 
-  have been changed to use this API, and the extractTerms() method has 
+  on it.  All locations in the code that called Weight.extractTerms()
+  have been changed to use this API, and the extractTerms() method has
   been deprecated. (Alan Woodward, Simon Willnauer, David Smiley, Luca
   Cavanna)
 
@@ -1493,9 +1495,9 @@ Improvements
 * LUCENE-8732: ConstantScoreQuery can now early terminate the query if the minimum score is
   greater than the constant score and total hits are not requested. (Jim Ferenczi)
 
-* LUCENE-8750: Implements setMissingValue() on sort fields produced from 
+* LUCENE-8750: Implements setMissingValue() on sort fields produced from
   DoubleValuesSource and LongValuesSource (Mike Sokolov via Alan Woodward)
-  
+
 * LUCENE-8701: ToParentBlockJoinQuery now creates a child scorer that disallows skipping over
   non-competitive documents if the score of a parent depends on the score of multiple
   children (avg, max, min). Additionally the score mode `none` that assigns a constant score to
@@ -1601,7 +1603,7 @@ API Changes
 * LUCENE-8373: StandardAnalyzer.ENGLISH_STOP_WORD_SET has been removed
   (Alan Woodward)
 
-* LUCENE-8388: Unused PostingsEnum#attributes() method has been removed 
+* LUCENE-8388: Unused PostingsEnum#attributes() method has been removed
   (Alan Woodward)
 
 * LUCENE-8405: TopDocs.maxScore is removed. IndexSearcher and TopFieldCollector
@@ -1645,7 +1647,7 @@ API Changes
   (Mayya Sharipova, Alan Woodward)
 
 * LUCENE-8597: IntervalIterator now exposes a gaps() method that reports the
-  number of gaps between its component sub-intervals.  This can be used in a 
+  number of gaps between its component sub-intervals.  This can be used in a
   new filter available via Intervals.maxgaps().  (Alan Woodward)
 
 * LUCENE-8609: Remove IndexWriter#numDocs() and IndexWriter#maxDoc() in favor
@@ -1744,10 +1746,10 @@ New Features
 * LUCENE-8646: New interval functions: Intervals.prefix() and Intervals.wildcard()
   (Alan Woodward)
 
-* LUCENE-8655: Add a getter in FunctionScoreQuery class in order to access to the 
+* LUCENE-8655: Add a getter in FunctionScoreQuery class in order to access to the
   underlying DoubleValuesSource. (Gérald Quaire via Alan Woodward)
 
-* LUCENE-8697: GraphTokenStreamFiniteStrings correctly handles side paths 
+* LUCENE-8697: GraphTokenStreamFiniteStrings correctly handles side paths
   containing gaps (Alan Woodward)
 
 * LUCENE-8702: Simplify intervals returned from vararg Intervals factory methods
@@ -1848,11 +1850,11 @@ Bug fixes
 Changes in Runtime Behavior
 
 * LUCENE-8527: StandardTokenizer and UAX29URLEmailTokenizer now support Unicode 9.0,
-  and provide Unicode UTS#51 v11.0 Emoji tokenization with the "<EMOJI>" token type. 
+  and provide Unicode UTS#51 v11.0 Emoji tokenization with the "<EMOJI>" token type.
 
 Build
 
-* LUCENE-8611: Update randomizedtesting to 2.7.2, JUnit to 4.12, add hamcrest-core 
+* LUCENE-8611: Update randomizedtesting to 2.7.2, JUnit to 4.12, add hamcrest-core
   dependency. (Dawid Weiss)
 
 * LUCENE-8537: ant test command fails under lucene/tools (Peter Somogyi)
@@ -1865,7 +1867,7 @@ Bug fixes:
 * LUCENE-8603: Fix the inversion of right ids for additional nouns in the Korean user dictionary.
   (Yoo Jeongin via Jim Ferenczi)
 
-* LUCENE-8624: int overflow in ByteBuffersDataOutput.size(). (Mulugeta Mammo, 
+* LUCENE-8624: int overflow in ByteBuffersDataOutput.size(). (Mulugeta Mammo,
   Dawid Weiss)
 
 * LUCENE-8625: int overflow in ByteBuffersDataInput.sliceBufferList. (Mulugeta Mammo,
@@ -1921,7 +1923,7 @@ Improvements
 
 * LUCENE-8581: Change LatLonShape encoding to use 4 bytes Per Dimension.
   (Ignacio Vera, Nick Knize, Adrien Grand)
-  
+
 * LUCENE-8527: Upgrade JFlex dependency to 1.7.0; in StandardTokenizer and UAX29URLEmailTokenizer,
   increase supported Unicode version from 6.3 to 9.0, and support Unicode UTS#51 v11.0 Emoji tokenization.
 
@@ -1939,7 +1941,7 @@ Optimizations
 * LUCENE-8598: Moved to the default accepted overhead ratio for packet ints in DocValuesFieldUpdats
   yields an up-to 4x performance improvement when applying doc values updates. (Simon Willnauer, Adrien Grand)
 
-* LUCENE-8599: Use sparse bitset to store docs in SingleValueDocValuesFieldUpdates. 
+* LUCENE-8599: Use sparse bitset to store docs in SingleValueDocValuesFieldUpdates.
   (Simon Willnauer, Adrien Grand)
 
 * LUCENE-8600: Doc-value updates get applied faster by sorting with quicksort,
@@ -1951,7 +1953,7 @@ Optimizations
 Test Framework
 
 * LUCENE-8604: TestRuleLimitSysouts now has an optional "hard limit" of bytes that can be written
-  to stderr and stdout (anything beyond the hard limit is ignored). The default hard limit is 2 GB of 
+  to stderr and stdout (anything beyond the hard limit is ignored). The default hard limit is 2 GB of
   logs per test class. (Dawid Weiss)
 
 Other
@@ -2032,7 +2034,7 @@ New Features
   Rectangle) from WKT format. (Nick Knize)
 
 * LUCENE-8462: Adds an Arabic snowball stemmer based on
-  https://github.com/snowballstem/snowball/blob/master/algorithms/arabic.sbl 
+  https://github.com/snowballstem/snowball/blob/master/algorithms/arabic.sbl
   (Ryadh Dahimene via Jim Ferenczi)
 
 * LUCENE-8554: Add new LatLonShapeLineQuery that queries indexed LatLonShape fields
@@ -2146,7 +2148,7 @@ Bug Fixes
   via LUCENE-8165 since we are now checking for offset+length going out of bounds.
   (Robert Muir, Nhat Nyugen, Simon Willnauer)
 
-* LUCENE-8370: Reproducing 
+* LUCENE-8370: Reproducing
   TestLucene{54,70}DocValuesFormat.testSortedSetVariableLengthBigVsStoredFields()
   failures (Erick Erickson)
 
@@ -2155,7 +2157,7 @@ Bug Fixes
   not set position increment in end() (Alan Woodward)
 
 * LUCENE-8395: WordDelimiterGraphFilter would incorrectly insert a hole into a
-  TokenStream if a token consisting entirely of delimiter characters was 
+  TokenStream if a token consisting entirely of delimiter characters was
   encountered, but preserve_original was set. (Alan Woodward)
 
 * LUCENE-8398: TieredMergePolicy.getMaxMergedSegmentMB has rounding error (Erick Erickson)
@@ -2172,7 +2174,7 @@ Bug Fixes
 * LUCENE-8466: IndexWriter.deleteDocs(Query... query) incorrectly applies deletes on flush
   if the index is sorted. (Adrien Grand, Jim Ferenczi, Vish Ramachandran)
 
-* LUCENE-8502: Allow access to delegate in FilterCodecReader. FilterCodecReader didn't 
+* LUCENE-8502: Allow access to delegate in FilterCodecReader. FilterCodecReader didn't
   allow access to it's delegate like other filter readers. This adds a new #getDelegate method
   to access the wrapped reader. (Simon Willnauer)
 
@@ -2185,8 +2187,8 @@ Changes in Runtime Behavior
   aggressively by default ensuring that no more than ~1/3 of the index size is
   used by deleted documents. (Adrien Grand)
 
-* LUCENE-8503: Call #getDelegate instead of direct member access during unwrap. 
-  Filter*Reader instances access the member or the delegate directly instead of 
+* LUCENE-8503: Call #getDelegate instead of direct member access during unwrap.
+  Filter*Reader instances access the member or the delegate directly instead of
   calling getDelegate(). In order to track access of the delegate these methods
   should call #getDelegate() (Simon Willnauer)
 
@@ -2229,7 +2231,7 @@ Improvements
 
 Build
 
-* LUCENE-5143: Stop publishing KEYS file with each version, use topmost lucene/KEYS file only. 
+* LUCENE-5143: Stop publishing KEYS file with each version, use topmost lucene/KEYS file only.
   The buildAndPushRelease.py script validates that RM's PGP key is in the KEYS file.
   Remove unused 'copy-to-stage' and '-dist-keys' targets from ant build. (janhoy)
 
@@ -2261,8 +2263,8 @@ Other
 ======================= Lucene 7.4.1 =======================
 
 Bug Fixes
- 
- * LUCENE-8365: Fix ArrayIndexOutOfBoundsException in UnifiedHighlighter. This fixes 
+
+ * LUCENE-8365: Fix ArrayIndexOutOfBoundsException in UnifiedHighlighter. This fixes
    a "off by one" error in the UnifiedHighlighter's code that is only triggered when
    two nested SpanNearQueries contain the same term. (Marc-Andre Morissette via Simon Willnauer)
 
@@ -2299,7 +2301,7 @@ Changes in Runtime Behavior
 
 * LUCENE-8309: Live docs are no longer backed by a FixedBitSet. (Adrien Grand)
 
-* LUCENE-8330: Detach IndexWriter from MergePolicy. MergePolicy now instead of 
+* LUCENE-8330: Detach IndexWriter from MergePolicy. MergePolicy now instead of
   requiring IndexWriter as a hard dependency expects a MergeContext which
   IndexWriter implements. (Simon Willnauer, Robert Muir, Dawid Weiss, Mike McCandless)
 
@@ -2307,7 +2309,7 @@ New Features
 
 * LUCENE-8200: Allow doc-values to be updated atomically together
   with a document. Doc-Values updates now can be used as a soft-delete
-  mechanism to all keeping several version of a document or already 
+  mechanism to all keeping several version of a document or already
   deleted documents around for later reuse. See "IW.softUpdateDocument(...)"
   for reference. (Simon Willnauer)
 
@@ -2323,7 +2325,7 @@ New Features
   searches based on minimum-interval semantics. (Alan Woodward, Adrien Grand,
   Jim Ferenczi, Simon Willnauer, Matt Weber)
 
-* LUCENE-8233: Add support for soft deletes to IndexWriter delete accounting. 
+* LUCENE-8233: Add support for soft deletes to IndexWriter delete accounting.
   Soft deletes are accounted for inside the index writer and therefor also
   by merge policies. A SoftDeletesRetentionMergePolicy is added that allows
   to selectively carry over soft_deleted document across merges for retention
@@ -2333,12 +2335,12 @@ New Features
   soft deletes if the reader is opened form a directory. (Simon Willnauer,
   Mike McCandless, Uwe Schindler, Adrien Grand)
 
-* LUCENE-8229, LUCENE-8270: Add a method Weight.matches(LeafReaderContext, doc) 
+* LUCENE-8229, LUCENE-8270: Add a method Weight.matches(LeafReaderContext, doc)
   that returns an iterator over matching positions for a given query and document.
-  This allows exact hit extraction and will enable implementation of accurate 
+  This allows exact hit extraction and will enable implementation of accurate
   highlighters. (Alan Woodward, Adrien Grand, David Smiley)
 
-* LUCENE-8249: Implement Matches API for phrase queries (Alan Woodward, Adrien 
+* LUCENE-8249: Implement Matches API for phrase queries (Alan Woodward, Adrien
   Grand)
 
 * LUCENE-8246: Allow to customize the number of deletes a merge claims. This
@@ -2353,19 +2355,19 @@ New Features
   marked with KeywordAttribute (Mike Sokolov via Mike McCandless)
 
 * LUCENE-8297: Add IW#tryUpdateDocValues(Reader, int, Fields...) IndexWriter can
-  update doc values for a specific term but this might affect all documents 
-  containing the term. With tryUpdateDocValues users can update doc-values 
+  update doc values for a specific term but this might affect all documents
+  containing the term. With tryUpdateDocValues users can update doc-values
   fields for individual documents. This allows for instance to soft-delete
   individual documents. (Simon Willnauer)
 
 * LUCENE-8298: Allow DocValues updates to reset a value. Passing a DV field with a null
-  value to IW#updateDocValues or IW#tryUpdateDocValues will now remove the value from the 
+  value to IW#updateDocValues or IW#tryUpdateDocValues will now remove the value from the
   provided document. This allows to undelete a soft-deleted document unless it's been claimed
-  by a merge. (Simon Willnauer) 
+  by a merge. (Simon Willnauer)
 
 * LUCENE-8273: ConditionalTokenFilter allows analysis chains to skip particular token
   filters based on the attributes of the current token. This generalises the keyword
-  token logic currently used for stemmers and WDF.  It is integrated into 
+  token logic currently used for stemmers and WDF.  It is integrated into
   CustomAnalyzer by using the `when` and `whenTerm` builder methods, and a new
   ProtectedTermFilter is added as an example.  (Alan Woodward, Robert Muir,
   David Smiley, Steve Rowe, Mike Sokolov)
@@ -2376,7 +2378,7 @@ New Features
   IndexFileDeleter already accounts for that for existing files which we can
   now use to also take pending deletes into account which ensures that all file
   generations per segment always go forward. (Simon Willnauer)
-  
+
 * LUCENE-7960: Add preserveOriginal option to the NGram and EdgeNGram filters.
   (Ingomar Wesp, Shawn Heisey via Robert Muir)
 
@@ -2407,8 +2409,8 @@ Bug Fixes
   index file names for updated doc values fields (Simon Willnauer,
   Michael McCandless, Nhat Nguyen)
 
-* LUCENE-8275: Push up #checkPendingDeletes to Directory to ensure IW fails if 
-  the directory has pending deletes files even if the directory is filtered or 
+* LUCENE-8275: Push up #checkPendingDeletes to Directory to ensure IW fails if
+  the directory has pending deletes files even if the directory is filtered or
   a FileSwitchDirectory (Simon Willnauer, Robert Muir)
 
 * LUCENE-8244: Do not leak open file descriptors in SearcherTaxonomyManager's
@@ -2422,10 +2424,10 @@ Bug Fixes
   (Julie Tibshirani via Jim Ferenczi)
 
 * LUCENE-8317: Prevent concurrent deletes from being applied during full flush.
-  Future deletes could potentially be exposed to flushes/commits/refreshes if the 
+  Future deletes could potentially be exposed to flushes/commits/refreshes if the
   amount of RAM used by deletes is greater than half of the IW RAM buffer. (Simon Willnauer)
 
-* LUCENE-8320: Fix WindowsFS to correctly account for rename and hardlinks. 
+* LUCENE-8320: Fix WindowsFS to correctly account for rename and hardlinks.
   (Simon Willnauer, Nhat Nguyen)
 
 * LUCENE-8328: Ensure ReadersAndUpdates consistently executes under lock.
@@ -2472,7 +2474,7 @@ Other
 
 * LUCENE-8214: Improve selection of testPoint for GeoComplexPolygon.
   (Ignacio Vera)
-  
+
 * SOLR-10912: Add automatic patch validation. (Mano Kovacs, Steve Rowe)
 
 * LUCENE-8122, LUCENE-8175: Upgrade analysis/icu to ICU 61.1.
@@ -2689,7 +2691,7 @@ New Features
 
 * LUCENE-8061: Add convenience factory methods to create BBoxes and XYZSolids
   directly from bounds objects.
-  
+
 * LUCENE-7736: IndexReaderFunctions expose various IndexReader statistics as
   DoubleValuesSources. (Alan Woodward)
 
@@ -2740,7 +2742,7 @@ Bug Fixes
 
 * LUCENE-8048: Filesystems do not guarantee order of directories updates
   (Nikolay Martynov, Simon Willnauer, Erick Erickson)
-  
+
 Optimizations
 
 * LUCENE-8018: Smaller FieldInfos memory footprint by not retaining unnecessary
@@ -2786,11 +2788,11 @@ New Features
 * LUCENE-7970: Add a shape to Geo3D that consists of multiple planes that
   approximate a true circle, rather than an ellipse, for non-spherical planet models.
   (Karl Wright, Ignacio Vera)
-  
+
 * LUCENE-7955: Add support for the concept of "nearest distance" to Geo3D's
   GeoPath abstraction, which is the distance along the path to the point that is
   closest to the provided point. (Karl Wright)
-  
+
 * LUCENE-7906: Add spatial relationships between all currently-defined Geo shapes.
   (Ignacio Vera)
 
@@ -2824,7 +2826,7 @@ New Features
 
 * LUCENE-7973: Update dictionary version for Ukrainian analyzer to 3.9.0 (Andriy
   Rysin via Dawid Weiss)
-  
+
 * LUCENE-7974: Add FloatPointNearestNeighbor, an N-dimensional FloatPoint
   K-nearest-neighbor search implementation.  (Steve Rowe)
 
@@ -2857,7 +2859,7 @@ Optimizations
 * LUCENE-7939: MinShouldMatchSumScorer now leverages two-phase iteration in
   order to be faster when used in conjunctions. (Adrien Grand)
 
-* LUCENE-7827: AnalyzingInfixSuggester doesn't create "textgrams" 
+* LUCENE-7827: AnalyzingInfixSuggester doesn't create "textgrams"
   when minPrefixChar=0 (Mikhail Khludnev)
 
 Bug Fixes
@@ -2898,15 +2900,15 @@ Build
 * SOLR-11181: Switch order of maven artifact publishing procedure: deploy first
   instead of locally installing first, to workaround a double repository push of
   *-sources.jar and *-javadoc.jar files.  (Lynn Monson via Steve Rowe)
-  
-* LUCENE-6673: Maven build fails for target javadoc:jar.  
+
+* LUCENE-6673: Maven build fails for target javadoc:jar.
   (Ramkumar Aiyengar, Daniel Collins via Steve Rowe)
 
 * LUCENE-7985: Upgrade forbiddenapis to 2.4.1.  (Uwe Schindler)
 
 Other
 
-* LUCENE-7948, LUCENE-7937: Upgrade randomizedtesting to 2.5.3 (minor fixes 
+* LUCENE-7948, LUCENE-7937: Upgrade randomizedtesting to 2.5.3 (minor fixes
   in test filtering for IDEs). (Mike Sokolov, Dawid Weiss)
 
 * LUCENE-7933: LongBitSet now validates the numBits parameter (Won
@@ -2917,7 +2919,7 @@ Other
 
 * LUCENE-7983: IndexWriter.IndexReaderWarmer is now a functional interface
   instead of an abstract class with a single method (Dawid Weiss)
-  
+
 * LUCENE-5753: Update TLDs recognized by UAX29URLEmailTokenizer. (Steve Rowe)
 
 
@@ -3044,7 +3046,7 @@ Bug Fixes
 * LUCENE-7859: Spatial-extras PackedQuadPrefixTree bug that only revealed itself
   with the new pointsOnly optimizations in LUCENE-7845. (David Smiley)
 
-* LUCENE-7871: fix false positive match in BlockJoinSelector when children have no value, introducing 
+* LUCENE-7871: fix false positive match in BlockJoinSelector when children have no value, introducing
   wrap methods accepting children as DISI. Extracting ToParentDocValues (Mikhail Khludnev)
 
 * LUCENE-7914: Add a maximum recursion level in automaton recursive
@@ -3204,7 +3206,7 @@ Bug Fixes
 * LUCENE-7808: Fixed PayloadScoreQuery and SpanPayloadCheckQuery
   .equals and .hashCode methods.  (Erik Hatcher)
 
-* LUCENE-7798: Add .equals and .hashCode to ToParentBlockJoinSortField 
+* LUCENE-7798: Add .equals and .hashCode to ToParentBlockJoinSortField
   (Mikhail Khludnev)
 
 * LUCENE-7814: DateRangePrefixTree (in spatial-extras) had edge-case bugs for
@@ -3213,7 +3215,7 @@ Bug Fixes
 * LUCENE-5365, LUCENE-7818: Fix incorrect condition in queryparser's
   QueryNodeOperation#logicalAnd().  (Olivier Binda, Amrit Sarkar,
   AppChecker via Uwe Schindler)
-  
+
 * LUCENE-7821: The classic and flexible query parsers, as well as Solr's
  "lucene"/standard query parser, should require " TO " in range queries,
   and accept "TO" as endpoints in range queries. (hossman, Steve Rowe)
@@ -3260,10 +3262,10 @@ Optimizations
 
 Other
 
-* LUCENE-7796: Make IOUtils.reThrow idiom declare Error return type so 
-  callers may use it in a way that compiler knows subsequent code is 
+* LUCENE-7796: Make IOUtils.reThrow idiom declare Error return type so
+  callers may use it in a way that compiler knows subsequent code is
   unreachable. reThrow is now deprecated in favor of IOUtils.rethrowAlways
-  with a slightly different semantics (see javadoc). (Hossman, Robert Muir, 
+  with a slightly different semantics (see javadoc). (Hossman, Robert Muir,
   Dawid Weiss)
 
 * LUCENE-7754: Inner classes should be static whenever possible.
@@ -3341,9 +3343,9 @@ API Changes
   Mike McCandless)
 
 * LUCENE-7700: A cleanup of merge throughput control logic. Refactored all the
-  code previously scattered throughout the IndexWriter and 
-  ConcurrentMergeScheduler into a more accessible set of public methods (see 
-  MergePolicy.OneMergeProgress, MergeScheduler.wrapForMerge and 
+  code previously scattered throughout the IndexWriter and
+  ConcurrentMergeScheduler into a more accessible set of public methods (see
+  MergePolicy.OneMergeProgress, MergeScheduler.wrapForMerge and
   OneMerge.mergeInit). (Dawid Weiss, Mike McCandless).
 
 * LUCENE-7734: FieldType's copy constructor was widened to accept any IndexableFieldType.
@@ -3370,7 +3372,7 @@ New Features
   for best performance. (Adrien Grand)
 
 * LUCENE-7673: Added MultiValued[Int/Long/Float/Double]FieldSource that given a
-  SortedNumericSelector.Type can give a ValueSource view of a 
+  SortedNumericSelector.Type can give a ValueSource view of a
   SortedNumericDocValues field. (Tomás Fernández Löbbe)
 
 * LUCENE-7465: Add SimplePatternTokenizer and
@@ -3424,7 +3426,7 @@ Improvements
   (Adrien Grand)
 
 * LUCENE-7662: If index files are missing, throw CorruptIndexException instead
-  of the less descriptive FileNotFound or NoSuchFileException (Mike Drob via 
+  of the less descriptive FileNotFound or NoSuchFileException (Mike Drob via
   Mike McCandless, Erick Erickson)
 
 * LUCENE-7680: UsageTrackingQueryCachingPolicy never caches term filters anymore
@@ -3440,7 +3442,7 @@ Improvements
   Mike McCandless)
 
 * LUCENE-7695: ComplexPhraseQueryParser to support query time synonyms (Markus Jelsma
-  via Mikhail Khludnev) 
+  via Mikhail Khludnev)
 
 * LUCENE-7747: QueryBuilder now iterates lazily over the possible paths when building a graph query
   (Jim Ferenczi)
@@ -3488,7 +3490,7 @@ Build
 
 * LUCENE-7665: Remove grouping dependency from the join module.
   (Martijn van Groningen)
-  
+
 * SOLR-10023: Add non-recursive 'test-nocompile' target: Only runs unit tests.
   Jars are not downloaded; compilation is not updated; and Clover is not enabled.
   (Steve Rowe)
@@ -3563,7 +3565,7 @@ API Changes
   grouping Collectors are renamed to remove the Abstract* prefix.
   (Alan Woodward, Martijn van Groningen)
 
-* LUCENE-7609: The expressions module now uses the DoubleValuesSource API, and 
+* LUCENE-7609: The expressions module now uses the DoubleValuesSource API, and
   no longer depends on the queries module.  Expression#getValueSource() is
   replaced with Expression#getDoubleValuesSource(). (Alan Woodward, Adrien
   Grand)
@@ -3608,7 +3610,7 @@ New features
   concurrently across all segments in the index (Emmanuel Keller via
   Mike McCandless)
 
-* LUCENE-7627: Added .intersect methods to SortedDocValues and 
+* LUCENE-7627: Added .intersect methods to SortedDocValues and
   SortedSetDocValues to allow filtering their TermsEnums with a
   CompiledAutomaton (Alan Woodward, Mike McCandless)
 
@@ -3619,7 +3621,7 @@ Bug Fixes
 
 * LUCENE-7562: CompletionFieldsConsumer sometimes throws
   NullPointerException on ghost fields (Oliver Eilhard via Mike McCandless)
-  
+
 * LUCENE-7533: Classic query parser: disallow autoGeneratePhraseQueries=true
   when splitOnWhitespace=false (and vice-versa). (Steve Rowe)
 
@@ -3675,7 +3677,7 @@ Improvements
 
 * LUCENE-7524: Added more detailed explanation of how IDF is computed in
   ClassicSimilarity and BM25Similarity. (Adrien Grand)
-  
+
 * LUCENE-7564: AnalyzingInfixSuggester should close its IndexWriter by default
   at the end of build(). (Steve Rowe)
 
@@ -3710,7 +3712,7 @@ Improvements
 * LUCENE-7401: Changed the way BKD trees pick the split dimension in order to
   ensure all dimensions are indexed. (Adrien Grand)
 
-* LUCENE-7614: Complex Phrase Query parser ignores double quotes around single token 
+* LUCENE-7614: Complex Phrase Query parser ignores double quotes around single token
   prefix, wildcard, range queries (Mikhail Khludnev)
 
 * LUCENE-7620: Added LengthGoalBreakIterator, a wrapper around another B.I. to skip breaks
@@ -3757,7 +3759,7 @@ Build
   Lucene and Solr DOAP RDF files into the Git source repository under
   dev-tools/doap/ and then pulling release dates from those files, rather than
   from JIRA. (Mano Kovacs, hossman, Steve Rowe)
-  
+
 * LUCENE-7596: Update Groovy to version 2.4.8 to allow building with Java 9
   build 148+. Also update JGit version for working-copy checks. (Uwe Schindler)
 
@@ -3841,7 +3843,7 @@ Other
 
 * LUCENE-7513: Upgrade randomizedtesting to 2.4.0. (Dawid Weiss)
 
-* LUCENE-7452: Block join query exception suggests how to find a doc, which 
+* LUCENE-7452: Block join query exception suggests how to find a doc, which
  violates orthogonality requirement. (Mikhail Khludnev)
 
 * LUCENE-7438: Renovate the Benchmark module's support for benchmarking highlighting. All
@@ -4005,7 +4007,7 @@ Improvements
 * LUCENE-7385: Improve/fix assert messages in SpanScorer. (David Smiley)
 
 * LUCENE-7393: Add ICUTokenizer option to parse Myanmar text as syllables instead of words,
-  because the ICU word-breaking algorithm has some issues. This allows for the previous 
+  because the ICU word-breaking algorithm has some issues. This allows for the previous
   tokenization used before Lucene 5. (AM, Robert Muir)
 
 * LUCENE-7409: Changed MMapDirectory's unmapping to work safer, but still with
@@ -4046,7 +4048,7 @@ Other
   (Uwe Schindler)
 
 * LUCENE-7360: Explanation.toHtml() is deprecated. (Alan Woodward)
-  
+
 * LUCENE-7372: Factor out an org.apache.lucene.search.FilterWeight class.
   (Christine Poerschke, Adrien Grand, David Smiley)
 
@@ -4070,7 +4072,7 @@ New Features
   queries with excluded bounds. (Adrien Grand)
 
 * LUCENE-7300: The misc module now has a directory wrapper that uses hard-links if
-  applicable and supported when copying files from another FSDirectory in 
+  applicable and supported when copying files from another FSDirectory in
   Directory#copyFrom. (Simon Willnauer)
 
 API Changes
@@ -4099,7 +4101,7 @@ Optimizations
 * LUCENE-7071: Reduce bytes copying in OfflineSorter, giving ~10%
   speedup on merging 2D LatLonPoint values (Mike McCandless)
 
-* LUCENE-7105, LUCENE-7215: Optimize LatLonPoint's newDistanceQuery. 
+* LUCENE-7105, LUCENE-7215: Optimize LatLonPoint's newDistanceQuery.
   (Robert Muir)
 
 * LUCENE-7097: IntroSorter now recurses to 2 * log_2(count) quicksort
@@ -4174,7 +4176,7 @@ Other
 * LUCENE-7295: TermAutomatonQuery.hashCode calculates Automaton.toDot().hash,
   equivalence relationship replaced with object identity. (Dawid Weiss)
 
-* LUCENE-7277: Make Query.hashCode and Query.equals abstract. (Paul Elschot, 
+* LUCENE-7277: Make Query.hashCode and Query.equals abstract. (Paul Elschot,
   Dawid Weiss)
 
 * LUCENE-7174: Upgrade randomizedtesting to 2.3.4. (Uwe Schindler, Dawid Weiss)
@@ -4202,10 +4204,10 @@ Build
 * LUCENE-7292: Use '-release' instead of '-source/-target' during
   compilation on Java 9+ to ensure real cross-compilation.
   (Uwe Schindler)
-  
+
 * LUCENE-7296: Update forbiddenapis to version 2.1.
   (Uwe Schindler)
-  
+
 ======================= Lucene 6.0.1 =======================
 
 New Features
@@ -4269,8 +4271,8 @@ New Features
 * LUCENE-6631: Lucene Document classification (Tommaso Teofili, Alessandro Benedetti)
 
 * LUCENE-6747: FingerprintFilter is a TokenFilter that outputs a single
-  token which is a concatenation of the sorted and de-duplicated set of 
-  input tokens. Useful for normalizing short text in clustering/linking 
+  token which is a concatenation of the sorted and de-duplicated set of
+  input tokens. Useful for normalizing short text in clustering/linking
   tasks. (Mark Harwood, Adrien Grand)
 
 * LUCENE-5735: NumberRangePrefixTreeStrategy now includes interval/range faceting
@@ -4281,7 +4283,7 @@ New Features
   length computations, to avoid skew from documents that don't have the field.
   (Ahmet Arslan via Robert Muir)
 
-* LUCENE-6758: Use docCount+1 for DefaultSimilarity's IDF, so that queries 
+* LUCENE-6758: Use docCount+1 for DefaultSimilarity's IDF, so that queries
   containing nonexistent fields won't screw up querynorm. (Terry Smith, Robert Muir)
 
 * SOLR-7876: The QueryTimeout interface now has a isTimeoutEnabled method
@@ -4391,7 +4393,7 @@ API Changes
 
 Optimizations
 
-* LUCENE-6891: Use prefix coding when writing points in 
+* LUCENE-6891: Use prefix coding when writing points in
   each leaf block in the default codec, to reduce the index
   size (Mike McCandless)
 
@@ -4608,9 +4610,9 @@ Bug fixes
 
 New Features
 
-* LUCENE-5868: JoinUtil.createJoinQuery(..,NumericType,..) query-time join 
+* LUCENE-5868: JoinUtil.createJoinQuery(..,NumericType,..) query-time join
   for LONG and INT fields with NUMERIC and SORTED_NUMERIC doc values.
-  (Alexey Zelin via Mikhail Khludnev) 
+  (Alexey Zelin via Mikhail Khludnev)
 
 * LUCENE-6939: Add exponential reciprocal scoring to
   BlendedInfixSuggester, to even more strongly favor suggestions that
@@ -4620,21 +4622,21 @@ New Features
   as alternative to their SPI name. This enables compile-time safety when
   defining analyzer's components.  (Uwe Schindler, Shai Erera)
 
-* LUCENE-6818, LUCENE-6986: Add DFISimilarity implementing the divergence 
+* LUCENE-6818, LUCENE-6986: Add DFISimilarity implementing the divergence
   from independence model. (Ahmet Arslan via Robert Muir)
 
 * SOLR-4619: Added removeAllAttributes() to AttributeSource, which removes
   all previously added attributes.
-  
+
 * LUCENE-7010: Added MergePolicyWrapper to allow easy wrapping of other policies.
   (Shai Erera)
 
 API Changes
 
-* LUCENE-6997: refactor sandboxed GeoPointField and query classes to lucene-spatial 
+* LUCENE-6997: refactor sandboxed GeoPointField and query classes to lucene-spatial
   module under new lucene.spatial.geopoint package (Nick Knize)
 
-* LUCENE-6908: GeoUtils static relational methods have been refactored to new 
+* LUCENE-6908: GeoUtils static relational methods have been refactored to new
   GeoRelationUtils and now correctly handle large irregular rectangles, and
   pole crossing distance queries. (Nick Knize)
 
@@ -4770,7 +4772,7 @@ Other
   AnalysisSPILoader: Don't wrap exceptions occuring in factory's
   ctor inside InvocationTargetException.  (Uwe Schindler)
 
-* LUCENE-6965: Expression's JavascriptCompiler now throw ParseException 
+* LUCENE-6965: Expression's JavascriptCompiler now throw ParseException
   with bad function names or bad arity instead of IllegalArgumentException.
   (Tomás Fernández Löbbe, Uwe Schindler, Ryan Ernst)
 
@@ -4834,7 +4836,7 @@ Bug Fixes
 
 New Features
 
-* LUCENE-6875: New Serbian Filter. (Nikola Smolenski via Robert Muir, 
+* LUCENE-6875: New Serbian Filter. (Nikola Smolenski via Robert Muir,
   Dawid Weiss)
 
 * LUCENE-6720: New FunctionRangeQuery wrapper around ValueSourceScorer
@@ -4895,7 +4897,7 @@ API Changes
   (Adrien Grand)
 
 * LUCENE-6870: DisjunctionMaxQuery#add is now deprecated, clauses should all be
-  provided at construction time. (Adrien Grand) 
+  provided at construction time. (Adrien Grand)
 
 * LUCENE-6884: Analyzer.tokenStream() and Tokenizer.setReader() are no longer
   declared as throwing IOException. (Alan Woodward)
@@ -4963,17 +4965,17 @@ Optimizations
 * LUCENE-6909: Remove unnecessary synchronized from
   FacetsConfig.getDimConfig for better concurrency (Sanne Grinovero
   via Mike McCandless)
-  
+
 * SOLR-7730: Speed up SlowCompositeReaderWrapper.getSortedSetDocValues() by
   avoiding merging FieldInfos just to check doc value type.
   (Paul Vasilyev, Yuriy Pakhomov, Mikhail Khludnev, yonik)
 
 Bug Fixes
 
-* LUCENE-6905: Unwrap center longitude for dateline crossing 
+* LUCENE-6905: Unwrap center longitude for dateline crossing
   GeoPointDistanceQuery. (Nick Knize)
 
-* LUCENE-6817: ComplexPhraseQueryParser.ComplexPhraseQuery does not display 
+* LUCENE-6817: ComplexPhraseQueryParser.ComplexPhraseQuery does not display
   slop in toString(). (Ahmet Arslan via Dawid Weiss)
 
 * LUCENE-6730: Hyper-parameter c is ignored in term frequency NormalizationH1.
@@ -4989,7 +4991,7 @@ Bug Fixes
   (Adrien Grand)
 
 * LUCENE-6776: Fix geo3d math to handle randomly squashed planet
-  models (Karl Wright via Mike McCandless) 
+  models (Karl Wright via Mike McCandless)
 
 * LUCENE-6792: Fix TermsQuery.toString() to work with binary terms.
   (Ruslan Muzhikov, Robert Muir)
@@ -5051,10 +5053,10 @@ Other
 
 * LUCENE-6862: Upgrade of RandomizedRunner to version 2.2.0. (Dawid Weiss)
 
-* LUCENE-6857: Validate StandardQueryParser with NOT operator 
+* LUCENE-6857: Validate StandardQueryParser with NOT operator
   with-in parantheses. (Jigar Shah via Dawid Weiss)
 
-* LUCENE-6827: Use explicit capacity ArrayList instead of a LinkedList 
+* LUCENE-6827: Use explicit capacity ArrayList instead of a LinkedList
   in MultiFieldQueryNodeProcessor. (Dawid Weiss).
 
 * LUCENE-6812: Upgrade RandomizedTesting to 2.1.17. (Dawid Weiss)
@@ -5082,10 +5084,10 @@ Other
 * LUCENE-6761: MatchAllDocsQuery's Scorers do not expose approximations
   anymore. (Adrien Grand)
 
-* LUCENE-6775, LUCENE-6833: Improved MorfologikFilterFactory to allow 
-  loading of custom dictionaries from ResourceLoader. Upgraded 
+* LUCENE-6775, LUCENE-6833: Improved MorfologikFilterFactory to allow
+  loading of custom dictionaries from ResourceLoader. Upgraded
   Morfologik to version 2.0.1. The 'dictionary' attribute has been
-  reverted back and now points at the dictionary resource to be 
+  reverted back and now points at the dictionary resource to be
   loaded instead of the default Polish dictionary.
   (Uwe Schindler, Dawid Weiss)
 
@@ -5190,7 +5192,7 @@ Tests
 New Features
 
 * LUCENE-6485: Add CustomSeparatorBreakIterator to postings
-  highlighter which splits on any character. For example, it 
+  highlighter which splits on any character. For example, it
   can be used with getMultiValueSeparator render whole field
   values.  (Luca Cavanna via Robert Muir)
 
@@ -5310,9 +5312,9 @@ New Features
 
 API Changes
 
-* LUCENE-6508: Simplify Lock api, there is now just 
-  Directory.obtainLock() which returns a Lock that can be 
-  released (or fails with exception). Add lock verification 
+* LUCENE-6508: Simplify Lock api, there is now just
+  Directory.obtainLock() which returns a Lock that can be
+  released (or fails with exception). Add lock verification
   to IndexWriter. Improve exception messages when locking fails.
   (Uwe Schindler, Mike McCandless, Robert Muir)
 
@@ -5448,7 +5450,7 @@ Bug fixes
 
 * LUCENE-6681: SortingMergePolicy must override MergePolicy.size(...).
   (Christine Poerschke via Adrien Grand)
-  
+
 * LUCENE-6682: StandardTokenizer performance bug: scanner buffer is
   unnecessarily copied when maxTokenLength doesn't change.  Also stop silently
   maxing out buffer size (and effectively also max token length) at 1M chars,
@@ -5464,7 +5466,7 @@ Bug fixes
 
 * LUCENE-6704: GeoPointDistanceQuery was visiting too many term ranges,
   consuming too much heap for a large radius (Nick Knize via Mike McCandless)
-  
+
 * SOLR-5882: fix ScoreMode.Min at ToParentBlockJoinQuery (Mikhail Khludnev)
 
 * LUCENE-6718: JoinUtil.createJoinQuery failed to rewrite queries before
@@ -5492,9 +5494,9 @@ Changes in Runtime Behavior
 
 * LUCENE-6537: NearSpansOrdered no longer tries to minimize its
   Span matches.  This means that the matching algorithm is entirely
-  lazy.  All spans returned by the previous implementation are still 
-  reported, but matching documents may now also return additional 
-  spans that were previously discarded in preference to shorter 
+  lazy.  All spans returned by the previous implementation are still
+  reported, but matching documents may now also return additional
+  spans that were previously discarded in preference to shorter
   overlapping ones. (Alan Woodward, Adrien Grand, Paul Elschot)
 
 * LUCENE-6538: Also include java.vm.version and java.runtime.version
@@ -5529,7 +5531,7 @@ Changes in Runtime Behavior
 
 * LUCENE-6579: IndexWriter now sacrifices (closes) itself to protect the index
   when an unexpected, tragic exception strikes while merging. (Robert
-  Muir, Mike McCandless) 
+  Muir, Mike McCandless)
 
 * LUCENE-6691: SortingMergePolicy.isSorted now considers FilterLeafReader instances.
   EarlyTerminatingSortingCollector.terminatedEarly accessor added.
@@ -5579,7 +5581,7 @@ Optimizations
 * LUCENE-6621: Removed two unused variables in analysis/stempel/src/java/org/
   egothor/stemmer/Compile.java
   (Rishabh Patel via Christine Poerschke)
-    
+
 Build
 
 * LUCENE-6518: Don't report false thread leaks from IBM J9
@@ -5590,7 +5592,7 @@ Build
 
 * LUCENE-6568: Make rat invocation depend on ivy configuration being set up
   (Ramkumar Aiyengar)
-  
+
 * LUCENE-6683: ivy-fail goal directs people to non-existent page
   (Mike Drob via Steve Rowe)
 
@@ -5662,14 +5664,14 @@ Bug Fixes
 
 New Features
 
-* LUCENE-6308, LUCENE-6385, LUCENE-6391: Span queries now share 
+* LUCENE-6308, LUCENE-6385, LUCENE-6391: Span queries now share
   document conjunction/intersection
   code with boolean queries, and use two-phased iterators for
   faster intersection by avoiding loading positions in certain cases.
   (Paul Elschot, Terry Smith, Robert Muir via Mike McCandless)
-  
+
 * LUCENE-6393: Add two-phase support to SpanPositionCheckQuery
-  and its subclasses: SpanPositionRangeQuery, SpanPayloadCheckQuery, 
+  and its subclasses: SpanPositionRangeQuery, SpanPayloadCheckQuery,
   SpanNearPayloadCheckQuery, SpanFirstQuery. (Paul Elschot, Robert Muir)
 
 * LUCENE-6394: Add two-phase support to SpanNotQuery and refactor
@@ -5780,9 +5782,9 @@ Bug Fixes
 * LUCENE-6409: Fixed integer overflow in LongBitSet.ensureCapacity.
   (Luc Vanlerberghe via Adrien Grand)
 
-* LUCENE-6424, LUCENE-6430: Fix many bugs with mockfs filesystems in the 
-  test-framework: always consistently wrap Path, fix buggy behavior for 
-  globs, implement equals/hashcode for filtered Paths, etc.  
+* LUCENE-6424, LUCENE-6430: Fix many bugs with mockfs filesystems in the
+  test-framework: always consistently wrap Path, fix buggy behavior for
+  globs, implement equals/hashcode for filtered Paths, etc.
   (Ryan Ernst, Simon Willnauer, Robert Muir)
 
 * LUCENE-6426: Fix FieldType's copy constructor to also copy over the numeric
@@ -5845,7 +5847,7 @@ API Changes
 
 Other
 
-* LUCENE-6413: Test runner should report the number of suites completed/ 
+* LUCENE-6413: Test runner should report the number of suites completed/
   remaining. (Dawid Weiss)
 
 * LUCENE-5439: Add 'ant jacoco' build target. (Robert Muir)
@@ -5881,8 +5883,8 @@ Test Framework
 
 New Features
 
-* LUCENE-6066: Added DiversifiedTopDocsCollector to misc for collecting no more 
-  than a given number of results under a choice of key. Introduces new remove 
+* LUCENE-6066: Added DiversifiedTopDocsCollector to misc for collecting no more
+  than a given number of results under a choice of key. Introduces new remove
   method to core's PriorityQueue. (Mark Harwood)
 
 * LUCENE-6191: New spatial 2D heatmap faceting for PrefixTreeStrategy. (David Smiley)
@@ -5909,12 +5911,12 @@ New Features
 
 Bug Fixes
 
-* LUCENE-6368: FST.save can truncate output (BufferedOutputStream may be closed 
+* LUCENE-6368: FST.save can truncate output (BufferedOutputStream may be closed
   after the underlying stream). (Ippei Matsushima via Dawid Weiss)
 
-* LUCENE-6249: StandardQueryParser doesn't support pure negative clauses. 
+* LUCENE-6249: StandardQueryParser doesn't support pure negative clauses.
   (Dawid Weiss)
-  
+
 * LUCENE-6190: Spatial pointsOnly flag on PrefixTreeStrategy shouldn't switch all predicates to
   Intersects. (David Smiley)
 
@@ -5948,8 +5950,8 @@ Bug Fixes
 Optimizations
 
 * LUCENE-6183, LUCENE-5647: Avoid recompressing stored fields
-  and term vectors when merging segments without deletions. 
-  Lucene50Codec's BEST_COMPRESSION mode uses a higher deflate 
+  and term vectors when merging segments without deletions.
+  Lucene50Codec's BEST_COMPRESSION mode uses a higher deflate
   level for more compact storage.  (Robert Muir)
 
 * LUCENE-6184: Make BooleanScorer only score windows that contain
@@ -5981,7 +5983,7 @@ Optimizations
 
 * LUCENE-6241: FSDirectory.listAll() doesnt filter out subdirectories anymore,
   for faster performance. Subdirectories don't matter to Lucene. If you need to
-  filter out non-index files with some custom usage, you may want to look at 
+  filter out non-index files with some custom usage, you may want to look at
   the IndexFileNames class. (Robert Muir)
 
 * LUCENE-6262: ConstantScoreQuery does not wrap the inner weight anymore when
@@ -6020,7 +6022,7 @@ API Changes
 
 * LUCENE-4524, LUCENE-6246, LUCENE-6256, LUCENE-6271: Merge DocsEnum and DocsAndPositionsEnum
   into a single PostingsEnum iterator.  TermsEnum.docs() and TermsEnum.docsAndPositions()
-  are replaced by TermsEnum.postings(). 
+  are replaced by TermsEnum.postings().
   (Alan Woodward, Simon Willnauer, Robert Muir, Ryan Ernst)
 
 * LUCENE-6222: Removed TermFilter, use a QueryWrapperFilter(TermQuery)
@@ -6048,7 +6050,7 @@ API Changes
 * LUCENE-6289: Replace DocValuesRangeFilter with DocValuesRangeQuery which
   supports approximations. (Adrien Grand)
 
-* LUCENE-6266: Remove unnecessary Directory params from SegmentInfo.toString, 
+* LUCENE-6266: Remove unnecessary Directory params from SegmentInfo.toString,
   SegmentInfos.files/toString, and SegmentCommitInfo.toString. (Robert Muir)
 
 * LUCENE-6272: Scorer extends DocSetIdIterator rather than DocsEnum (Alan
@@ -6112,7 +6114,7 @@ New Features
 
 * LUCENE-5945: All file handling converted to NIO.2 apis. (Robert Muir)
 
-* LUCENE-5946: SimpleFSDirectory now uses Files.newByteChannel, for 
+* LUCENE-5946: SimpleFSDirectory now uses Files.newByteChannel, for
   portability with custom FileSystemProviders. If you want the old
   non-interruptible behavior of RandomAccessFile, use RAFDirectory
   in the misc/ module. (Uwe Schindler, Robert Muir)
@@ -6159,7 +6161,7 @@ New Features
   exiting requests that take too long to enumerate over terms. (Anshum Gupta, Steve Rowe,
   Robert Muir)
 
-* LUCENE-5911: Add MemoryIndex.freeze() to allow thread-safe searching over a 
+* LUCENE-5911: Add MemoryIndex.freeze() to allow thread-safe searching over a
   MemoryIndex. (Alan Woodward, David Smiley, Robert Muir)
 
 * LUCENE-5969: Lucene 5.0 has a new index format with mismatched file detection,
@@ -6239,7 +6241,7 @@ Optimizations
   (Adrien Grand)
 
 * LUCENE-5969: Refactor merging to be more efficient, checksum calculation is
-  per-segment/per-producer, and norms and doc values merging no longer cause 
+  per-segment/per-producer, and norms and doc values merging no longer cause
   RAM spikes for latent fields. (Mike McCandless, Robert Muir)
 
 * LUCENE-5983: CachingWrapperFilter now uses a new DocIdSet implementation
@@ -6262,7 +6264,7 @@ Optimizations
   implemented.  Only one of offsets or positions are required of the term vector.
   (David Smiley)
 
-* LUCENE-6089, LUCENE-6090: Tune CompressionMode.HIGH_COMPRESSION for 
+* LUCENE-6089, LUCENE-6090: Tune CompressionMode.HIGH_COMPRESSION for
   better compression and less cpu usage. (Adrien Grand, Robert Muir)
 
 * LUCENE-6034: QueryScorer, used by the default highlighter, needn't re-index the provided
@@ -6276,7 +6278,7 @@ Optimizations
 
 * LUCENE-6131: Optimize SortingMergePolicy. (Robert Muir)
 
-* LUCENE-6133: Improve default StoredFieldsWriter.merge() to be more efficient. 
+* LUCENE-6133: Improve default StoredFieldsWriter.merge() to be more efficient.
   (Robert Muir)
 
 * LUCENE-6145: Make EarlyTerminatingSortingCollector able to early-terminate
@@ -6298,7 +6300,7 @@ API Changes
   not positioned. This change affects all classes that inherit from
   DocIdSetIterator, including DocsEnum and DocsAndPositionsEnum. (Adrien Grand)
 
-* LUCENE-5127: Reduce RAM usage of FixedGapTermsIndex. Remove 
+* LUCENE-5127: Reduce RAM usage of FixedGapTermsIndex. Remove
   IndexWriterConfig.setTermIndexInterval, IndexWriterConfig.setReaderTermsIndexDivisor,
   and termsIndexDivisor from StandardDirectoryReader. These options have been no-ops
   with the default codec since Lucene 4.0. If you want to configure the interval for
@@ -6307,7 +6309,7 @@ API Changes
 
 * LUCENE-5388: Remove Reader from Tokenizer's constructor and from
   Analyzer's createComponents. TokenStreams now always get their input
-  via setReader.  
+  via setReader.
   (Benson Margulies via Robert Muir - pull request #16)
 
 * LUCENE-5527: The Collector API has been refactored to use a dedicated Collector
@@ -6330,10 +6332,10 @@ API Changes
   UninvertingReader is more efficient: supports multi-valued numeric fields,
   detects when a multi-valued field is single-valued, reuses caches
   of compatible types (e.g. SORTED also supports BINARY and SORTED_SET access
-  without insanity).  "Insanity" is no longer possible unless you explicitly want it. 
-  Rename FieldCache* and DocTermOrds* classes in the search package to DocValues*. 
+  without insanity).  "Insanity" is no longer possible unless you explicitly want it.
+  Rename FieldCache* and DocTermOrds* classes in the search package to DocValues*.
   Move SortedSetSortField to core and add SortedSetFieldSource to queries/, which
-  takes the same selectors. Add helper methods to DocValues.java that are better 
+  takes the same selectors. Add helper methods to DocValues.java that are better
   suited for search code (never return null, etc).  (Mike McCandless, Robert Muir)
 
 * LUCENE-5871: Remove Version from IndexWriterConfig. Use
@@ -6357,9 +6359,9 @@ API Changes
 * LUCENE-5924: Rename CheckIndex -fix option to -exorcise. This option does not
   actually fix the index, it just drops data.  (Robert Muir)
 
-* LUCENE-5969: Add Codec.compoundFormat, which handles the encoding of compound 
+* LUCENE-5969: Add Codec.compoundFormat, which handles the encoding of compound
   files. Add getMergeInstance() to codec producer APIs, which can be overridden
-  to return an instance optimized for merging instead of searching. Add 
+  to return an instance optimized for merging instead of searching. Add
   Terms.getStats() which can return additional codec-specific statistics about a field.
   Change instance method SegmentInfos.read() to two static methods: SegmentInfos.readCommit()
   and SegmentInfos.readLatestCommit().
@@ -6392,8 +6394,8 @@ API Changes
   instance based on a Directory implementation passed to the factory method.
   See MIGRATE.txt for more details.  (Uwe Schindler, Robert Muir)
 
-* LUCENE-6062: Throw exception instead of silently doing nothing if you try to 
-  sort/group/etc on a misconfigured field (e.g. no docvalues, no UninvertingReader, etc).  
+* LUCENE-6062: Throw exception instead of silently doing nothing if you try to
+  sort/group/etc on a misconfigured field (e.g. no docvalues, no UninvertingReader, etc).
   (Robert Muir)
 
 * LUCENE-6068: LeafReader.fields() never returns null. (Robert Muir)
@@ -6483,7 +6485,7 @@ Bug Fixes
 * LUCENE-6044: Fix backcompat support for token filters with enablePositionIncrements=false.
   Also fixed backcompat for TrimFilter with updateOffsets=true.  These options
   are supported with a match version before 4.4, and no longer valid at all with 5.0.
-  (Ryan Ernst) 
+  (Ryan Ernst)
 
 * LUCENE-6042: CustomScoreQuery explain was incorrect in some cases,
   such as when nested inside a boolean query. (Denis Lantsman via Robert Muir)
@@ -6525,21 +6527,21 @@ Bug Fixes
 
 * LUCENE-6124: Fix double-close() problems in codec and store APIs.
   (Robert Muir)
-  
+
 * LUCENE-6152: Fix double close problems in OutputStreamIndexOutput.
   (Uwe Schindler)
 
 * LUCENE-6139: Highlighter: TokenGroup start & end offset getters should have
   been returning the offsets of just the matching tokens in the group when
   there's a distinction. (David Smiley)
-  
+
 * LUCENE-6173: NumericTermAttribute and spatial/CellTokenStream do not clone
   their BytesRef(Builder)s. Also equals/hashCode was missing.  (Uwe Schindler)
 
 * LUCENE-6205: Fixed intermittent concurrency issue that could cause
   FileNotFoundException when writing doc values updates at the same
   time that a merge kicks off.  (Mike McCandless)
-  
+
 * LUCENE-6192: Fix int overflow corruption case in skip data for
   high frequency terms in extremely large indices (Robert Muir, Mike
   McCandless)
@@ -6571,7 +6573,7 @@ Tests
 * LUCENE-5974: Add check that backcompat indexes use default codecs
   (Ryan Ernst)
 
-* LUCENE-5971: Create addBackcompatIndexes.py script to build and add 
+* LUCENE-5971: Create addBackcompatIndexes.py script to build and add
   backcompat test indexes for a given lucene version. Also renamed backcompat
   index files to use Version.toString() in filename.
   (Ryan Ernst)
@@ -6610,7 +6612,7 @@ Other
 * LUCENE-4086: Removed support for Lucene 3.x indexes. See migration guide for
   more information.  (Robert Muir)
 
-* LUCENE-5858: Moved Lucene 4 compatibility codecs to 'lucene-backward-codecs.jar'. 
+* LUCENE-5858: Moved Lucene 4 compatibility codecs to 'lucene-backward-codecs.jar'.
   (Adrien Grand, Robert Muir)
 
 * LUCENE-5915: Remove Pulsing postings format. (Robert Muir)
@@ -6666,7 +6668,7 @@ Bug fixes
 * LUCENE-6001: DrillSideways hits NullPointerException for certain
   BooleanQuery searches.  (Dragan Jotannovic, jane chang via Mike
   McCandless)
-  
+
 * LUCENE-6306: Merging of doc values and norms now checks whether the
   merge was aborted so IndexWriter.rollback can more promptly abort a
   running merge. (Robert Muir, Mike McCandless)
@@ -6705,7 +6707,7 @@ Bug fixes
 
 * LUCENE-6075: Don't overflow int in SimpleRateLimiter (Boaz Leskes
   via Mike McCandless)
-  
+
 * LUCENE-5980: Don't let document length overflow. (Robert Muir)
 
 * LUCENE-6042: CustomScoreQuery explain was incorrect in some cases,
@@ -6719,7 +6721,7 @@ Bug fixes
 
 * LUCENE-6094: Allow IW.rollback to stop ConcurrentMergeScheduler even
   when it's stalling because there are too many merges. (Mike McCandless)
-  
+
 Documentation
 
 * LUCENE-6057: Improve Sort(SortField) docs (Martin Braun via Mike McCandless)
@@ -6753,7 +6755,7 @@ Bug fixes
   to enforce here); use simple string tokenizer.  (Ryan Ernst, Uwe Schindler,
   Robert Muir, Mike McCandless)
 
-* LUCENE-5958: Don't let exceptions during checkpoint corrupt the index. 
+* LUCENE-5958: Don't let exceptions during checkpoint corrupt the index.
   Refactor existing OOM handling too, so you don't need to handle OOM special
   for every IndexWriter method: instead such disasters will cause IW to close itself
   defensively. (Robert Muir, Mike McCandless)
@@ -6863,7 +6865,7 @@ API Changes
   to set the version an analyzer to replicate behavior from a specific release.
   (Ryan Ernst, Robert Muir)
 
-  
+
 Optimizations
 
 * LUCENE-5780: Make OrdinalMap more memory-efficient, especially in case the
@@ -6885,7 +6887,7 @@ Optimizations
   Solr's schema support.  (Shay Banon, Uwe Schindler, Robert Muir)
 
 * LUCENE-5795: MoreLikeThisQuery now only collects the top N terms instead
-  of collecting all terms from the like text when building the query. 
+  of collecting all terms from the like text when building the query.
   (Alex Ksikes, Simon Willnauer)
 
 * LUCENE-5681: Fix RAMDirectory's IndexInput to not do double buffering
@@ -6901,7 +6903,7 @@ Optimizations
 * LUCENE-5841: Improve performance of block tree terms dictionary when
   assigning terms to blocks.  (Mike McCandless)
 
-* LUCENE-5856: Optimize Fixed/Open/LongBitSet to remove unnecessary AND. 
+* LUCENE-5856: Optimize Fixed/Open/LongBitSet to remove unnecessary AND.
   (Robert Muir)
 
 * LUCENE-5884: Optimize FST.ramBytesUsed.  (Adrien Grand, Robert Muir,
@@ -6917,17 +6919,17 @@ Optimizations
 
 Bug Fixes
 
-* LUCENE-5796: Fixes the Scorer.getChildren() method for two combinations 
+* LUCENE-5796: Fixes the Scorer.getChildren() method for two combinations
   of BooleanQuery. (Terry Smith via Robert Muir)
 
-* LUCENE-5790: Fix compareTo in MutableValueDouble and MutableValueBool, this caused 
-  incorrect results when grouping on fields with missing values. 
+* LUCENE-5790: Fix compareTo in MutableValueDouble and MutableValueBool, this caused
+  incorrect results when grouping on fields with missing values.
   (海老澤 志信, hossman)
 
 * LUCENE-5817: Fix hunspell zero-affix handling: previously only zero-strips worked
   correctly.  (Robert Muir)
 
-* LUCENE-5818, LUCENE-5823: Fix hunspell overgeneration for short strings that also 
+* LUCENE-5818, LUCENE-5823: Fix hunspell overgeneration for short strings that also
   match affixes, words are only stripped to a zero-length string if FULLSTRIP option
   is specified in the dictionary.  (Robert Muir)
 
@@ -6940,9 +6942,9 @@ Bug Fixes
 
 * LUCENE-5672: IndexWriter.addIndexes() calls maybeMerge(), to ensure the index stays
   healthy. If you don't want merging use NoMergePolicy instead. (Robert Muir)
-  
+
 * LUCENE-5908: Fix Lucene43NGramTokenizer to be final
-  
+
 Test Framework
 
 * LUCENE-5786: Unflushed/ truncated events file (hung testing subprocess).
@@ -6953,13 +6955,13 @@ Test Framework
   Ryan Ernst, Dawid Weiss)
 
 Build
- 
+
 * LUCENE-5770: Upgrade to JFlex 1.6, which has direct support for
   supplementary code points - as a result, ICU4J is no longer used
   to generate surrogate pairs to augment JFlex scanner specifications.
   (Steve Rowe)
 
-* SOLR-6358: Remove VcsDirectoryMappings from idea configuration 
+* SOLR-6358: Remove VcsDirectoryMappings from idea configuration
   vcs.xml (Ramkumar Aiyengar via Steve Rowe)
 
 ======================= Lucene 4.9.1 ======================
@@ -6997,7 +6999,7 @@ Bug fixes
   UAX29URLEmailTokenizer tokenize extremely slowly over long sequences of
   text partially matching certain grammar rules.  The scanner default
   buffer size was reduced, and scanner buffer growth was disabled, resulting
-  in much, much faster tokenization for these text sequences.  
+  in much, much faster tokenization for these text sequences.
   (Chris Geeringh, Robert Muir, Steve Rowe)
 
 ======================= Lucene 4.9.0 =======================
@@ -7009,7 +7011,7 @@ Changes in Runtime Behavior
   IndexWriter will now throw IllegalArgumentException.  (Robert Muir,
   Mike McCandless)
 
-* LUCENE-5646: Remove rare/undertested bulk merge algorithm in 
+* LUCENE-5646: Remove rare/undertested bulk merge algorithm in
   CompressingStoredFieldsWriter. (Robert Muir, Adrien Grand)
 
 New Features
@@ -7022,7 +7024,7 @@ New Features
 * LUCENE-5675: Add IDVersionPostingsFormat, a postings format
   optimized for primary-key (ID) fields that also record a version
   (long) for each ID.  (Robert Muir, Mike McCandless)
-  
+
 * LUCENE-5680: Add ability to atomically update a set of DocValues
   fields. (Shai Erera)
 
@@ -7031,7 +7033,7 @@ New Features
   (Luca Cavanna via Robert Muir)
 
 * LUCENE-5731, LUCENE-5760: Add RandomAccessInput, a random access API for directory.
-  Add DirectReader/Writer, optimized for reading packed integers directly 
+  Add DirectReader/Writer, optimized for reading packed integers directly
   from Directory. Add Lucene49Codec and Lucene49DocValuesFormat that make
   use of these.  (Robert Muir)
 
@@ -7100,7 +7102,7 @@ API Changes
 * LUCENE-5640: The Token class was deprecated. Since Lucene 2.9, TokenStreams
   are using Attributes, Token is no longer used.  (Uwe Schindler, Robert Muir)
 
-* LUCENE-5679: Consolidated IndexWriter.deleteDocuments(Term) and 
+* LUCENE-5679: Consolidated IndexWriter.deleteDocuments(Term) and
   IndexWriter.deleteDocuments(Query) with their varargs counterparts.
   (Shai Erera)
 
@@ -7116,7 +7118,7 @@ API Changes
 * LUCENE-5708: Remove IndexWriterConfig.clone, so now IndexWriter
   simply uses the IndexWriterConfig you pass it, and you must create a
   new IndexWriterConfig for each IndexWriter.  (Mike McCandless)
-  
+
 * LUCENE-5678: IndexOutput no longer allows seeking, so it is no longer required
   to use RandomAccessFile to write Indexes. Lucene now uses standard FileOutputStream
   wrapped with OutputStreamIndexOutput to write index data. BufferedIndexOutput was
@@ -7141,10 +7143,10 @@ Optimizations
 
 * LUCENE-5603: hunspell stemmer more efficiently strips prefixes
   and suffixes.  (Robert Muir)
-  
+
 * LUCENE-5599: HttpReplicator did not properly delegate bulk read() to wrapped
   InputStream. (Christoph Kaser via Shai Erera)
-  
+
 * LUCENE-5591: pass an IOContext with estimated flush size when applying DV
   updates. (Shai Erera)
 
@@ -7167,7 +7169,7 @@ Optimizations
 * LUCENE-4236: Optimize BooleanQuery's in-order scoring. This speeds up
   some types of boolean queries.  (Robert Muir)
 
-* LUCENE-5694: Don't score() subscorers in DisjunctionSumScorer or 
+* LUCENE-5694: Don't score() subscorers in DisjunctionSumScorer or
   DisjunctionMaxScorer unless score() is called.  (Robert Muir)
 
 * LUCENE-5720: Optimize DirectPackedReader's decompression. (Robert Muir)
@@ -7179,19 +7181,19 @@ Optimizations
 * LUCENE-5730: FSDirectory.open returns MMapDirectory for 64-bit operating
   systems, not just Linux and Windows. (Robert Muir)
 
-* LUCENE-5703: BinaryDocValues producers don't allocate or copy bytes on 
+* LUCENE-5703: BinaryDocValues producers don't allocate or copy bytes on
   each access anymore.  (Adrien Grand)
 
 * LUCENE-5721: Monotonic compression doesn't use zig-zag encoding anymore.
   (Robert Muir, Adrien Grand)
 
-* LUCENE-5750: Speed up monotonic addressing for BINARY and SORTED_SET 
+* LUCENE-5750: Speed up monotonic addressing for BINARY and SORTED_SET
   docvalues. (Robert Muir)
 
 * LUCENE-5751: Speed up MemoryDocValues. (Adrien Grand, Robert Muir)
 
 * LUCENE-5767: OrdinalMap optimizations, that mostly help on low cardinalities.
-  (Martijn van Groningen, Adrien Grand) 
+  (Martijn van Groningen, Adrien Grand)
 
 * LUCENE-5769: SingletonSortedSetDocValues now supports random access ordinals.
   (Robert Muir)
@@ -7202,7 +7204,7 @@ Bug fixes
   lock if the lock is already obtained by the JVM. Trying to obtain an already
   obtained lock in the same JVM can unlock the file might allow other processes
   to lock the file even without explicitly unlocking the FileLock. This behavior
-  is operating system dependent. (Simon Willnauer) 
+  is operating system dependent. (Simon Willnauer)
 
 * LUCENE-5673: MMapDirectory: Work around a "bug" in the JDK that throws
   a confusing OutOfMemoryError wrapped inside IOException if the FileChannel
@@ -7217,7 +7219,7 @@ Bug fixes
   if the underlying TermsEnum supports ord() and the insertion point would
   be at the end. (Robert Muir)
 
-* LUCENE-5618, LUCENE-5636: SegmentReader referenced unneeded files following 
+* LUCENE-5618, LUCENE-5636: SegmentReader referenced unneeded files following
   doc-values updates. Now doc-values field updates are written in separate file
   per field. (Shai Erera, Robert Muir)
 
@@ -7246,21 +7248,21 @@ Bug fixes
 
 Test Framework
 
-* LUCENE-5622: Fail tests if they print over the given limit of bytes to 
+* LUCENE-5622: Fail tests if they print over the given limit of bytes to
   System.out or System.err. (Robert Muir, Dawid Weiss)
-  
+
 * LUCENE-5619: Added backwards compatibility tests to ensure we can update existing
   indexes with doc-values updates. (Shai Erera, Robert Muir)
-  
+
 Build
-  
+
 * LUCENE-5442: The Ant check-lib-versions target now runs Ivy resolution
   transitively, then fails the build when it finds a version conflict: when a
   transitive dependency's version is more recent than the direct dependency's
   version specified in lucene/ivy-versions.properties.  Exceptions are
   specifiable in lucene/ivy-ignore-conflicts.properties.
   (Steve Rowe)
-  
+
 * LUCENE-5715: Upgrade direct dependencies known to be older than transitive
   dependencies: com.sun.jersey.version:1.8->1.9; com.sun.xml.bind:jaxb-impl:2.2.2->2.2.3-1;
   commons-beanutils:commons-beanutils:1.7.0->1.8.3; commons-digester:commons-digester:2.0->2.1;
@@ -7281,7 +7283,7 @@ Bug fixes
 
 * LUCENE-5599: HttpReplicator did not properly delegate bulk read() to wrapped
   InputStream. (Christoph Kaser via Shai Erera)
-  
+
 * LUCENE-5600: HttpClientBase did not properly consume a connection if a server
   error occurred. (Christoph Kaser via Shai Erera)
 
@@ -7308,7 +7310,7 @@ Bug fixes
   with more than 256 values would throw exception.
   (Mikhail Khludnev via Robert Muir)
 
-* LUCENE-5654: Fix various close() methods that could suppress 
+* LUCENE-5654: Fix various close() methods that could suppress
   throwables such as OutOfMemoryError, instead returning scary messages
   that look like index corruption.  (Mike McCandless, Robert Muir)
 
@@ -7369,12 +7371,12 @@ New Features
   (Robert Muir)
 
 * LUCENE-5493: SortingMergePolicy, and EarlyTerminatingSortingCollector
-  support arbitrary Sort specifications.  
+  support arbitrary Sort specifications.
   (Robert Muir, Mike McCandless, Adrien Grand)
-  
-* LUCENE-3758: Allow the ComplexPhraseQueryParser to search order or 
+
+* LUCENE-3758: Allow the ComplexPhraseQueryParser to search order or
   un-order proximity queries. (Ahmet Arslan via Erick Erickson)
-  
+
 * LUCENE-5530: ComplexPhraseQueryParser throws ParseException for fielded queries.
   (Erick Erickson via Tomas Fernandez Lobbe and Ahmet Arslan)
 
@@ -7409,9 +7411,9 @@ New Features
 * LUCENE-5558: Add TruncateTokenFilter which truncates terms to
   the specified length.  (Ahmet Arslan via Robert Muir)
 
-* LUCENE-2446: Added checksums to lucene index files. As of 4.8, the last 8 
+* LUCENE-2446: Added checksums to lucene index files. As of 4.8, the last 8
   bytes of each file contain a zlib-crc32 checksum. Small metadata files are
-  verified on load. Larger files can be checked on demand via 
+  verified on load. Larger files can be checked on demand via
   AtomicReader.checkIntegrity. You can configure this to happen automatically
   before merges by enabling IndexWriterConfig.setCheckIntegrityAtMerge.
   (Robert Muir)
@@ -7439,9 +7441,9 @@ API Changes
 * LUCENE-5468: Move offline Sort (from suggest module) to OfflineSort. (Robert Muir)
 
 * LUCENE-5493: SortingMergePolicy and EarlyTerminatingSortingCollector take
-  Sort instead of Sorter. BlockJoinSorter is removed, replaced with 
+  Sort instead of Sorter. BlockJoinSorter is removed, replaced with
   BlockJoinComparatorSource, which can take a Sort for ordering of parents
-  and a separate Sort for ordering of children within a block. 
+  and a separate Sort for ordering of children within a block.
   (Robert Muir, Mike McCandless, Adrien Grand)
 
 * LUCENE-5516: MergeScheduler#merge() now accepts a MergeTrigger as well as
@@ -7480,7 +7482,7 @@ API Changes
 Optimizations
 
 * LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads
-  all known openoffice dictionaries without error, and supports an additional 
+  all known openoffice dictionaries without error, and supports an additional
   longestOnly option for a less aggressive approach.  (Robert Muir)
 
 * LUCENE-4848: Use Java 7 NIO2-FileChannel instead of RandomAccessFile
@@ -7491,7 +7493,7 @@ Optimizations
 * LUCENE-5515: Improved TopDocs#merge to create a merged ScoreDoc
   array with length of at most equal to the specified size instead of length
   equal to at most from + size as was before. (Martijn van Groningen)
-  
+
 * LUCENE-5529: Spatial search of non-point indexed shapes should be a little
   faster due to skipping intersection tests on redundant cells. (David Smiley)
 
@@ -7529,7 +7531,7 @@ Bug fixes
 
 * LUCENE-5612: NativeFSLockFactory no longer deletes its lock file. This cannot be done
   safely without the risk of deleting someone else's lock file. If you use NativeFSLockFactory,
-  you may see write.lock hanging around from time to time: it's harmless.  
+  you may see write.lock hanging around from time to time: it's harmless.
   (Uwe Schindler, Mike McCandless, Robert Muir)
 
 * LUCENE-5624: Ensure NativeFSLockFactory does not leak file handles if it is unable
@@ -7544,7 +7546,7 @@ Bug fixes
   (Robert Muir)
 
 Tests
-  
+
 * LUCENE-5630: Fix TestAllAnalyzersHaveFactories to correctly check for existence
   of class and corresponding Map<String,String> ctor.  (Uwe Schindler, Robert Muir)
 
@@ -7555,7 +7557,7 @@ Test Framework
 * LUCENE-5577: Temporary folder and file management (and cleanup facilities)
   (Mark Miller, Uwe Schindler, Dawid Weiss)
 
-* LUCENE-5567: When a suite fails with zombie threads failure marker and count 
+* LUCENE-5567: When a suite fails with zombie threads failure marker and count
   is not propagated properly. (Dawid Weiss)
 
 * LUCENE-5449: Rename _TestUtil and _TestHelper to remove the leading _.
@@ -7580,7 +7582,7 @@ Build
 
 Documentation
 
-* LUCENE-5534: Add javadocs to GreekStemmer methods. 
+* LUCENE-5534: Add javadocs to GreekStemmer methods.
   (Stamatis Pitsios via Robert Muir)
 
 ======================= Lucene 4.7.2 =======================
@@ -7607,8 +7609,8 @@ Changes in Runtime Behavior
 
 Bug Fixes
 
-* LUCENE-5450: Fix getField() NPE issues with SpanOr/SpanNear when they have an 
-  empty list of clauses. This can happen for example,  when a wildcard matches 
+* LUCENE-5450: Fix getField() NPE issues with SpanOr/SpanNear when they have an
+  empty list of clauses. This can happen for example,  when a wildcard matches
   no terms.  (Tim Allison via Robert Muir)
 
 * LUCENE-5473: Throw IllegalArgumentException, not
@@ -7623,8 +7625,8 @@ Bug Fixes
 
 * LUCENE-5502: Fixed TermsFilter.equals that could return true for different
   filters. (Igor Motov via Adrien Grand)
-  
-* LUCENE-5522: FacetsConfig didn't add drill-down terms for association facet 
+
+* LUCENE-5522: FacetsConfig didn't add drill-down terms for association facet
   fields labels. (Shai Erera)
 
 * LUCENE-5520: ToChildBlockJoinQuery would hit
@@ -7650,8 +7652,8 @@ Bug Fixes
   indexed shapes within 1/2 maxDistErr from the edge of the query shape.  This meant
   searching for a point by the same point as a query rarely worked.  (David Smiley)
 
-* LUCENE-5553: IndexReader#ReaderClosedListener is not always invoked when 
-  IndexReader#close() is called or if refCount is 0. If an exception is 
+* LUCENE-5553: IndexReader#ReaderClosedListener is not always invoked when
+  IndexReader#close() is called or if refCount is 0. If an exception is
   thrown during internal close or on any of the close listeners some or all
   listeners might be missed. This can cause memory leaks if the core listeners
   are used to clear caches. (Simon Willnauer)
@@ -7667,7 +7669,7 @@ New Features
 
 * LUCENE-5336: Add SimpleQueryParser: parser for human-entered queries.
   (Jack Conradson via Robert Muir)
-  
+
 * LUCENE-5337: Add Payload support to FileDictionary (Suggest) and make it more
   configurable (Areek Zillur via Erick Erickson)
 
@@ -7676,7 +7678,7 @@ New Features
   (missing the term, weight or payload).  (Areek Zillur via
   Mike McCandless)
 
-* LUCENE-5404: Add .getCount method to all suggesters (Lookup); persist count 
+* LUCENE-5404: Add .getCount method to all suggesters (Lookup); persist count
   metadata on .store(); Dictionary returns InputIterator; Dictionary.getWordIterator
   renamed to .getEntryIterator. (Areek Zillur)
 
@@ -7703,7 +7705,7 @@ New Features
   Mike McCandless)
 
 * LUCENE-5099: QueryNode should have the ability to detach from its node
-  parent. Added QueryNode.removeFromParent() that allows nodes to be 
+  parent. Added QueryNode.removeFromParent() that allows nodes to be
   detached from its parent node. (Adriano Crestani)
 
 * LUCENE-5395 LUCENE-5451: Upgrade to Spatial4j 0.4.1: Parses WKT (including
@@ -7716,10 +7718,10 @@ New Features
 * LUCENE-5415: Add multitermquery (wildcards,prefix,etc) to PostingsHighlighter.
   (Mike McCandless, Robert Muir)
 
-* LUCENE-3069: Add two memory resident dictionaries (FST terms dictionary and 
-  FSTOrd terms dictionary) to improve primary key lookups. The PostingsBaseFormat 
-  API is also changed so that term dictionaries get the ability to block 
-  encode term metadata, and all dictionary implementations can now plug in any 
+* LUCENE-3069: Add two memory resident dictionaries (FST terms dictionary and
+  FSTOrd terms dictionary) to improve primary key lookups. The PostingsBaseFormat
+  API is also changed so that term dictionaries get the ability to block
+  encode term metadata, and all dictionary implementations can now plug in any
   PostingsBaseFormat. (Han Jiang, Mike McCandless)
 
 * LUCENE-5353: ShingleFilter's filler token should be configurable.
@@ -7736,7 +7738,7 @@ New Features
   Mike McCandless)
 
 * LUCENE-5434: NRT support for file systems that do no have delete on last
-  close or cannot delete while referenced semantics. 
+  close or cannot delete while referenced semantics.
   (Mark Miller, Mike McCandless)
 
 * LUCENE-5418: Drilling down or sideways on a Lucene facet range
@@ -7784,7 +7786,7 @@ Build
   Unicode 6.3; update UAX29URLEmailTokenizer's recognized top level
   domains in URLs and Emails from the IANA Root Zone Database.
   (Steve Rowe)
-  
+
 * LUCENE-5360: Add support for developing in Netbeans IDE.
   (Michal Hlavac, Uwe Schindler, Steve Rowe)
 
@@ -7862,12 +7864,12 @@ Bug fixes
   long (up to targetMaxStaleSec) when a searcher is waiting for a
   specific generation, when it should have waited for at most
   targetMinStaleSec. (Hans Lund via Mike McCandless)
-  
+
 API Changes
 
 * LUCENE-5339: The facet module was simplified/reworked to make the
-  APIs more approachable to new users. Note: when migrating to the new 
-  API, you must pass the Document that is returned from FacetConfig.build() 
+  APIs more approachable to new users. Note: when migrating to the new
+  API, you must pass the Document that is returned from FacetConfig.build()
   to IndexWriter.addDocument(). (Shai Erera, Gilad Barkai, Rob
   Muir, Mike McCandless)
 
@@ -7877,7 +7879,7 @@ API Changes
   etc. but it's easy to override the parseShape method if you wish. (David
   Smiley)
 
-* LUCENE-5414: DocumentExpressionDictionary was renamed to 
+* LUCENE-5414: DocumentExpressionDictionary was renamed to
   DocumentValueSourceDictionary and all dependencies to the lucene-expression
   module were removed from lucene-suggest. DocumentValueSourceDictionary now
   only accepts a ValueSource instead of a convenience ctor for an expression
@@ -7887,10 +7889,10 @@ API Changes
   responsible for encoding/decoding a block of terms.  Instead, they
   should encode/decode each term to/from a long[] and byte[].  (Han
   Jiang, Mike McCandless)
-  
-* LUCENE-5425: FacetsCollector and MatchingDocs use a general DocIdSet, 
+
+* LUCENE-5425: FacetsCollector and MatchingDocs use a general DocIdSet,
   allowing for custom implementations to be used when faceting.
-  (John Wang, Lei Wang, Shai Erera) 
+  (John Wang, Lei Wang, Shai Erera)
 
 Optimizations
 
@@ -7905,17 +7907,17 @@ Optimizations
 
 Changes in Runtime Behavior
 
-* LUCENE-5362: IndexReader and SegmentCoreReaders now throw 
+* LUCENE-5362: IndexReader and SegmentCoreReaders now throw
   AlreadyClosedException if the refCount in incremented but
   is less that 1. (Simon Willnauer)
 
 Documentation
 
-* LUCENE-5384: Add some tips for making tokenfilters and tokenizers 
-  to the analysis package overview.  
+* LUCENE-5384: Add some tips for making tokenfilters and tokenizers
+  to the analysis package overview.
   (Benson Margulies via Robert Muir - pull request #12)
 
-* LUCENE-5389: Add more guidance in the analysis documentation 
+* LUCENE-5389: Add more guidance in the analysis documentation
   package overview.
   (Benson Margulies via Robert Muir - pull request #14)
 
@@ -7968,7 +7970,7 @@ New Features
   McCandless)
 
 * LUCENE-5207, LUCENE-5334: Added expressions module for customizing ranking
-  with script-like syntax. 
+  with script-like syntax.
   (Jack Conradson, Ryan Ernst, Uwe Schindler via Robert Muir)
 
 * LUCENE-5180: ShingleFilter now creates shingles with trailing holes,
@@ -8027,24 +8029,24 @@ New Features
 * LUCENE-5189: Add IndexWriter.updateNumericDocValues, to update
   numeric DocValues fields of documents, without re-indexing them.
   (Shai Erera, Mike McCandless, Robert Muir)
-  
+
 * LUCENE-5298: Add SumValueSourceFacetRequest for aggregating facets by
   a ValueSource, such as a NumericDocValuesField or an expression.
   (Shai Erera)
 
 * LUCENE-5323: Add .sizeInBytes method to all suggesters (Lookup).
   (Areek Zillur via Mike McCandless)
-  
+
 * LUCENE-5312: Add BlockJoinSorter, a new Sorter implementation that makes sure
   to never split up blocks of documents indexed with IndexWriter.addDocuments.
   (Adrien Grand)
-  
+
 * LUCENE-5297: Allow to range-facet on any ValueSource, not just
   NumericDocValues fields. (Shai Erera)
-  
+
 Bug Fixes
 
-* LUCENE-5272: OpenBitSet.ensureCapacity did not modify numBits, causing 
+* LUCENE-5272: OpenBitSet.ensureCapacity did not modify numBits, causing
   false assertion errors in fastSet. (Shai Erera)
 
 * LUCENE-5303: OrdinalsCache did not use coreCacheKey, resulting in
@@ -8054,7 +8056,7 @@ Bug Fixes
   inside ConstantScoreQuery, which now rewrites to a query removing the
   obsolete QueryWrapperFilter.  (Adrien Grand, Uwe Schindler)
 
-* LUCENE-5330: IndexWriter didn't process all internal events on 
+* LUCENE-5330: IndexWriter didn't process all internal events on
   #getReader(), #close() and #rollback() which causes files to be
   deleted at a later point in time. This could cause short-term disk
   pollution or OOM if in-memory directories are used. (Simon Willnauer)
@@ -8113,7 +8115,7 @@ Optimizations
 
 Documentation
 
-* LUCENE-5211: Better javadocs and error checking of 'format' option in 
+* LUCENE-5211: Better javadocs and error checking of 'format' option in
   StopFilterFactory, as well as comments in all snowball formatted files
   about specifying format option.  (hossman)
 
@@ -8134,12 +8136,12 @@ Changes in backwards compatibility policy
 
 Build
 
-* LUCENE-5283: Fail the build if ant test didn't execute any tests 
+* LUCENE-5283: Fail the build if ant test didn't execute any tests
   (everything filtered out). (Dawid Weiss, Uwe Schindler)
 
 * LUCENE-5249, LUCENE-5257: All Lucene/Solr modules should use the same
   dependency versions. (Steve Rowe)
-  
+
 * LUCENE-5273: Binary artifacts in Lucene and Solr convenience binary
   distributions accompanying a release, including on Maven Central,
   should be identical across all distributions. (Steve Rowe, Uwe Schindler,
@@ -8213,14 +8215,14 @@ New features
 * LUCENE-5091: SpanNotQuery can now be configured with pre and post slop to act
   as a hypothetical SpanNotNearQuery. (Tim Allison via David Smiley)
 
-* LUCENE-4985: FacetsAccumulator.create() is now able to create a 
+* LUCENE-4985: FacetsAccumulator.create() is now able to create a
   MultiFacetsAccumulator over a mixed set of facet requests. MultiFacetsAccumulator
   allows wrapping multiple FacetsAccumulators, allowing to easily mix
   existing and custom ones. TaxonomyFacetsAccumulator supports any
   FacetRequest which implements createFacetsAggregator and was indexed
   using the taxonomy index. (Shai Erera)
 
-* LUCENE-5153: AnalyzerWrapper.wrapReader allows wrapping the Reader given to 
+* LUCENE-5153: AnalyzerWrapper.wrapReader allows wrapping the Reader given to
   inputReader. (Shai Erera)
 
 * LUCENE-5155: FacetRequest.getValueOf and .getFacetArraysSource replaced by
@@ -8230,13 +8232,13 @@ New features
 * LUCENE-5165: Add SuggestStopFilter, to be used with analyzing
   suggesters, so that a stop word at the very end of the lookup query,
   and without any trailing token characters, will be preserved.  This
-  enables query "a" to suggest apple; see 
+  enables query "a" to suggest apple; see
   http://blog.mikemccandless.com/2013/08/suggeststopfilter-carefully-removes.html
   for details.
 
 * LUCENE-5178: Added support for missing values to DocValues fields.
   AtomicReader.getDocsWithField returns a Bits of documents with a value,
-  and FieldCache.getDocsWithField forwards to that for DocValues fields. Things like 
+  and FieldCache.getDocsWithField forwards to that for DocValues fields. Things like
   SortField.setMissingValue, FunctionValues.exists, and FieldValueFilter now
   work with DocValues fields.  (Robert Muir)
 
@@ -8321,7 +8323,7 @@ Bug Fixes
 * LUCENE-5218: In some cases, trying to retrieve or merge a 0-length
   binary doc value would hit an ArrayIndexOutOfBoundsException.
   (Littlestar via Mike McCandless)
-  
+
 API Changes
 
 * LUCENE-5094: Add ramBytesUsed() to MultiDocValues.OrdinalMap.
@@ -8330,8 +8332,8 @@ API Changes
 * LUCENE-5114: Remove unused boolean useCache parameter from
   TermsEnum.seekCeil and .seekExact (Mike McCandless)
 
-* LUCENE-5128: IndexSearcher.searchAfter throws IllegalArgumentException if 
-  searchAfter exceeds the number of documents in the reader. 
+* LUCENE-5128: IndexSearcher.searchAfter throws IllegalArgumentException if
+  searchAfter exceeds the number of documents in the reader.
   (Crocket via Shai Erera)
 
 * LUCENE-5129: CategoryAssociationsContainer no longer supports null
@@ -8339,7 +8341,7 @@ API Changes
   associations, you should add them using FacetFields. (Shai Erera)
 
 * LUCENE-4876: IndexWriter no longer clones the given IndexWriterConfig. If you
-  need to use the same config more than once, e.g. when sharing between multiple 
+  need to use the same config more than once, e.g. when sharing between multiple
   writers, make sure to clone it before passing to each writer.
   (Shai Erera, Mike McCandless)
 
@@ -8350,9 +8352,9 @@ API Changes
   FacetRequest.createAggregator was replaced by OldFacetsAccumulator.createAggregator.
   (Shai Erera)
 
-* LUCENE-5149: CommonTermsQuery now allows to set the minimum number of terms that 
+* LUCENE-5149: CommonTermsQuery now allows to set the minimum number of terms that
   should match for its high and low frequent sub-queries. Previously this was only
-  supported on the low frequent terms query. (Simon Willnauer)  
+  supported on the low frequent terms query. (Simon Willnauer)
 
 * LUCENE-5156: CompressingTermVectors TermsEnum no longer supports ord().
   (Robert Muir)
@@ -8384,7 +8386,7 @@ API Changes
 
 Changes in Runtime Behavior
 
-* LUCENE-5178: DocValues codec consumer APIs (iterables) return null values 
+* LUCENE-5178: DocValues codec consumer APIs (iterables) return null values
   when the document has no value for the field. (Robert Muir)
 
 * LUCENE-5200: The HighFreqTerms command-line tool returns the true top-N
@@ -8396,7 +8398,7 @@ Optimizations
 * LUCENE-5088: Added TermFilter to filter docs by a specific term.
   (Martijn van Groningen)
 
-* LUCENE-5119: DiskDV keeps the document-to-ordinal mapping on disk for 
+* LUCENE-5119: DiskDV keeps the document-to-ordinal mapping on disk for
   SortedDocValues.  (Robert Muir)
 
 * LUCENE-5145: New AppendingPackedLongBuffer, a new variant of the former
@@ -8425,7 +8427,7 @@ Optimizations
   processed from within the IndexWriter in order to prevent situations
   where DWPT or DW calling int IW causing deadlocks. (Simon Willnauer)
 
-* LUCENE-5182: Terminate phrase searches early if max phrase window is 
+* LUCENE-5182: Terminate phrase searches early if max phrase window is
   exceeded in FastVectorHighlighter to prevent very long running phrase
   extraction if phrase terms are high frequent. (Simon Willnauer)
 
@@ -8437,7 +8439,7 @@ Optimizations
   DocIdSet implementation and uses WAH8DocIdSet by default, which should be
   more memory efficient than FixedBitSet on average as well as faster on small
   sets. (Robert Muir)
-  
+
 Documentation
 
 * LUCENE-4894: remove facet userguide as it was outdated. Partially absorbed into
@@ -8452,7 +8454,7 @@ Changes in backwards compatibility policy
   CheckIndex.fixIndex(Status). If you used to pass a codec to this method, just
   remove it from the arguments. (Adrien Grand)
 
-* LUCENE-5089, SOLR-5126: Update to Morfologik 1.7.1. MorfologikAnalyzer and MorfologikFilter 
+* LUCENE-5089, SOLR-5126: Update to Morfologik 1.7.1. MorfologikAnalyzer and MorfologikFilter
   no longer support multiple "dictionaries" as there is only one dictionary available.
   (Dawid Weiss)
 
@@ -8469,7 +8471,7 @@ Changes in backwards compatibility policy
 
 Build
 
-* SOLR-5159: Manifest includes non-parsed maven variables. 
+* SOLR-5159: Manifest includes non-parsed maven variables.
   (Artem Karpenko via Steve Rowe)
 
 * LUCENE-5193: Add jar-src as top-level target to generate all Lucene and Solr
@@ -8533,13 +8535,13 @@ Changes in backwards compatibility policy
   DictionaryCompoundWordTokenFilter and HyphenationCompoundWordTokenFilter don't
   update offsets anymore. (Adrien Grand)
 
-* LUCENE-5015: SamplingAccumulator no longer corrects the counts of the sampled 
-  categories. You should set TakmiSampleFixer on SamplingParams if required (but 
+* LUCENE-5015: SamplingAccumulator no longer corrects the counts of the sampled
+  categories. You should set TakmiSampleFixer on SamplingParams if required (but
   notice that this means slower search). (Rob Audenaerde, Gilad Barkai, Shai Erera)
 
 * LUCENE-4933: Replace ExactSimScorer/SloppySimScorer with just SimScorer. Previously
   there were 2 implementations as a performance hack to support tableization of
-  sqrt(), but this caching is removed, as sqrt is implemented in hardware with modern 
+  sqrt(), but this caching is removed, as sqrt is implemented in hardware with modern
   jvms and it's faster not to cache.  (Robert Muir)
 
 * LUCENE-5038: MergePolicy now has a default implementation for useCompoundFile based
@@ -8559,10 +8561,10 @@ Changes in backwards compatibility policy
 
 Bug Fixes
 
-* LUCENE-4890: QueryTreeBuilder.getBuilder() only finds interfaces on the 
+* LUCENE-4890: QueryTreeBuilder.getBuilder() only finds interfaces on the
   most derived class. (Adriano Crestani)
 
-* LUCENE-4997: Internal test framework's tests are sensitive to previous 
+* LUCENE-4997: Internal test framework's tests are sensitive to previous
   test failures and tests.failfast. (Dawid Weiss, Shai Erera)
 
 * LUCENE-4955: NGramTokenizer now supports inputs larger than 1024 chars.
@@ -8573,7 +8575,7 @@ Bug Fixes
 
 * LUCENE-4972: DirectoryTaxonomyWriter created empty commits even if no changes
   were made. (Shai Erera, Michael McCandless)
-  
+
 * LUCENE-949: AnalyzingQueryParser can't work with leading wildcards.
   (Tim Allison, Robert Muir, Steve Rowe)
 
@@ -8585,7 +8587,7 @@ Bug Fixes
   in exception messages.  (Markus Jelsma via Robert Muir)
 
 * LUCENE-4992: Fix constructor of CustomScoreQuery to take FunctionQuery
-  for scoringQueries. Instead use QueryValueSource to safely wrap arbitrary 
+  for scoringQueries. Instead use QueryValueSource to safely wrap arbitrary
   queries and use them with CustomScoreQuery.  (John Wang, Robert Muir)
 
 * LUCENE-5016: SamplingAccumulator returned inconsistent label if asked to
@@ -8625,7 +8627,7 @@ Bug Fixes
   setter in ConcurrentMergePolicy: setMaxMergesAndThreads.  Previously these
   setters would not work unless you invoked them very carefully.
   (Robert Muir, Shai Erera)
-  
+
 * LUCENE-5068: QueryParserUtil.escape() does not escape forward slash.
   (Matias Holte via Steve Rowe)
 
@@ -8649,7 +8651,7 @@ Optimizations
 
 * LUCENE-4951: DrillSideways uses the new Scorer.cost() method to make
   better decisions about which scorer to use internally.  (Mike McCandless)
-  
+
 * LUCENE-4976: PersistentSnapshotDeletionPolicy writes its state to a
   single snapshots_N file, and no longer requires closing (Mike
   McCandless, Shai Erera)
@@ -8675,7 +8677,7 @@ New Features
 * LUCENE-5064: Added PagedMutable (internal), a paged extension of
   PackedInts.Mutable which allows for storing more than 2B values. (Adrien Grand)
 
-* LUCENE-4766: Added a PatternCaptureGroupTokenFilter that uses Java regexes to 
+* LUCENE-4766: Added a PatternCaptureGroupTokenFilter that uses Java regexes to
   emit multiple tokens one for each capture group in one or more patterns.
   (Simon Willnauer, Clinton Gormley)
 
@@ -8699,7 +8701,7 @@ New Features
 * LUCENE-4979: LiveFieldFields can work with any ReferenceManager, not
   just ReferenceManager<IndexSearcher> (Mike McCandless).
 
-* LUCENE-4975: Added a new Replicator module which can replicate index 
+* LUCENE-4975: Added a new Replicator module which can replicate index
   revisions between server and client. (Shai Erera, Mike McCandless)
 
 * LUCENE-5022: Added FacetResult.mergeHierarchies to merge multiple
@@ -8720,7 +8722,7 @@ New Features
 * LUCENE-5079: IndexWriter.hasUncommittedChanges() returns true if there are
   changes that have not been committed. (yonik, Mike McCandless, Uwe Schindler)
 
-* SOLR-4565: Extend NorwegianLightStemFilter and NorwegianMinimalStemFilter 
+* SOLR-4565: Extend NorwegianLightStemFilter and NorwegianMinimalStemFilter
   to handle "nynorsk" (Erlend Garåsen, janhoy via Robert Muir)
 
 * LUCENE-5087: Add getMultiValuedSeparator to PostingsHighlighter, for cases
@@ -8740,19 +8742,19 @@ API Changes
 * LUCENE-5077: Make it easier to use compressed norms. Lucene42NormsFormat takes
   an overhead parameter, so you can easily pass a different value other than
   PackedInts.FASTEST from your own codec.  (Robert Muir)
-  
+
 * LUCENE-5097: Analyzer now has an additional tokenStream(String fieldName,
   String text) method, so wrapping by StringReader for common use is no
   longer needed. This method uses an internal reusable reader, which was
   previously only used by the Field class.  (Uwe Schindler, Robert Muir)
-  
+
 * LUCENE-4542: HunspellStemFilter's maximum recursion level is now configurable.
   (Piotr, Rafał Kuć via Adrien Grand)
-  
+
 Build
 
-* LUCENE-4987: Upgrade randomized testing to version 2.0.10: 
-  Test framework may fail internally due to overly aggressive J9 optimizations. 
+* LUCENE-4987: Upgrade randomized testing to version 2.0.10:
+  Test framework may fail internally due to overly aggressive J9 optimizations.
   (Dawid Weiss, Shai Erera)
 
 * LUCENE-5043: The eclipse target now uses the containing directory for the
@@ -8769,14 +8771,14 @@ Build
 
 Tests
 
-* LUCENE-4901: TestIndexWriterOnJRECrash should work on any 
+* LUCENE-4901: TestIndexWriterOnJRECrash should work on any
   JRE vendor via Runtime.halt().
   (Mike McCandless, Robert Muir, Uwe Schindler, Rodrigo Trujillo, Dawid Weiss)
 
 Changes in runtime behavior
 
 * LUCENE-5038: New segments written by IndexWriter are now wrapped into CFS
-  by default. DocumentsWriterPerThread doesn't consult MergePolicy anymore 
+  by default. DocumentsWriterPerThread doesn't consult MergePolicy anymore
   to decide if a CFS must be written, instead IndexWriterConfig now has a
   property to enable / disable CFS for newly created segments. (Simon Willnauer)
 
@@ -8849,10 +8851,10 @@ Changes in backwards compatibility policy
   via Mike McCandless)
 
 * LUCENE-4822: KeywordTokenFilter is now an abstract class. Subclasses
-  need to implement #isKeyword() in order to mark terms as keywords. 
-  The existing functionality has been factored out into a new 
+  need to implement #isKeyword() in order to mark terms as keywords.
+  The existing functionality has been factored out into a new
   SetKeywordTokenFilter class. (Simon Willnauer, Uwe Schindler)
-  
+
 * LUCENE-4642: Remove Tokenizer's and subclasses' ctors taking
   AttributeSource. (Renaud Delbru, Uwe Schindler, Steve Rowe)
 
@@ -8896,7 +8898,7 @@ New Features
 
 * LUCENE-4815: DrillSideways now allows more than one FacetRequest per
   dimension (Mike McCandless)
-  
+
 * LUCENE-3918: IndexSorter has been ported to 4.3 API and now supports
   sorting documents by a numeric DocValues field, or reverse the order of
   the documents in the index. Additionally, apps can implement their own
@@ -8913,9 +8915,9 @@ New Features
   determine whether the last token was finished or not, so that a
   query "i " will no longer suggest "Isla de Muerta" for example.
   (Mike McCandless)
-  
+
 * LUCENE-4642: Add create(AttributeFactory) to TokenizerFactory and
-  subclasses with ctors taking AttributeFactory. 
+  subclasses with ctors taking AttributeFactory.
   (Renaud Delbru, Uwe Schindler, Steve Rowe)
 
 * LUCENE-4820: Add payloads to Analyzing/FuzzySuggester, to record an
@@ -8938,8 +8940,8 @@ New Features
   per-IndexReader-open to compute its ordinal map, but it requires no
   taxonomy index and it tie-breaks facet labels in an understandable
   (by Unicode sort order) way.  (Robert Muir, Mike McCandless)
-  
-* LUCENE-4843: Add LimitTokenPositionFilter: don't emit tokens with 
+
+* LUCENE-4843: Add LimitTokenPositionFilter: don't emit tokens with
   positions that exceed the configured limit.  (Steve Rowe)
 
 * LUCENE-4832: Add ToParentBlockJoinCollector.getTopGroupsWithAllChildDocs, to retrieve
@@ -8994,7 +8996,7 @@ New Features
 * LUCENE-4905: Made the maxPassages parameter per-field in PostingsHighlighter.
   (Robert Muir)
 
-* LUCENE-4897: Added TaxonomyReader.getChildren for traversing a category's 
+* LUCENE-4897: Added TaxonomyReader.getChildren for traversing a category's
   children. (Shai Erera)
 
 * LUCENE-4902: Added FilterDirectoryReader to allow easy filtering of a
@@ -9058,7 +9060,7 @@ Bug Fixes
   IndexDeletionPolicy and InfoStream in order to make an IndexWriterConfig and
   its clone fully independent. (Adrien Grand)
 
-* LUCENE-4893: Facet counts were multiplied as many times as 
+* LUCENE-4893: Facet counts were multiplied as many times as
   FacetsCollector.getFacetResults() is called. (Shai Erera)
 
 * LUCENE-4888: Fixed SloppyPhraseScorer, MultiDocs(AndPositions)Enum and
@@ -9109,7 +9111,7 @@ Bug Fixes
   the number of documents per chunk has been added to fix this issue.
   (Robert Muir, Adrien Grand)
 
-* LUCENE-4934: Fix minor equals/hashcode problems in facet/DrillDownQuery, 
+* LUCENE-4934: Fix minor equals/hashcode problems in facet/DrillDownQuery,
   BoostingQuery, MoreLikeThisQuery, FuzzyLikeThisQuery, and block join queries.
   (Robert Muir, Uwe Schindler)
 
@@ -9127,7 +9129,7 @@ Documentation
 
 Build
 
-* LUCENE-4879: Upgrade randomized testing to version 2.0.9: 
+* LUCENE-4879: Upgrade randomized testing to version 2.0.9:
   Filter stack traces on console output. (Dawid Weiss, Robert Muir)
 
 
@@ -9156,7 +9158,7 @@ Bug Fixes
 * LUCENE-4828: BooleanQuery no longer extracts terms from its MUST_NOT
   clauses.  (Mike McCandless)
 
-* SOLR-4589: Fixed CPU spikes and poor performance in lazy field loading 
+* SOLR-4589: Fixed CPU spikes and poor performance in lazy field loading
   of multivalued fields. (hossman)
 
 * LUCENE-4870: Fix bug where an entire index might be deleted by the IndexWriter
@@ -9183,21 +9185,21 @@ Optimizations
 
 Changes in backwards compatibility policy
 
-* LUCENE-4602: FacetFields now stores facet ordinals in a DocValues field, 
+* LUCENE-4602: FacetFields now stores facet ordinals in a DocValues field,
   rather than a payload. This forces rebuilding existing indexes, or do a
-  one time migration using FacetsPayloadMigratingReader. Since DocValues 
+  one time migration using FacetsPayloadMigratingReader. Since DocValues
   support in-memory caching, CategoryListCache was removed too.
   (Shai Erera, Michael McCandless)
 
 * LUCENE-4697: FacetResultNode is now a concrete class with public members
   (instead of getter methods). (Shai Erera)
-  
+
 * LUCENE-4600: FacetsCollector is now an abstract class with two
-  implementations: StandardFacetsCollector (the old version of 
+  implementations: StandardFacetsCollector (the old version of
   FacetsCollector) and CountingFacetsCollector. FacetsCollector.create()
   returns the most optimized collector for the given parameters.
   (Shai Erera, Michael McCandless)
-  
+
 * LUCENE-4700: OrdinalPolicy is now per CategoryListParams, and is no longer
   an interface, but rather an enum with values NO_PARENTS and ALL_PARENTS.
   PathPolicy was removed, you should extend FacetFields and DrillDownStream
@@ -9217,11 +9219,11 @@ Changes in backwards compatibility policy
   (Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
 
 * LUCENE-4757: Cleanup and refactoring of FacetsAccumulator, FacetRequest,
-  FacetsAggregator and FacetResultsHandler API. If your application did 
+  FacetsAggregator and FacetResultsHandler API. If your application did
   FacetsCollector.create(), you should not be affected, but if you wrote
   an Aggregator, then you should migrate it to the per-segment
   FacetsAggregator. You can still use StandardFacetsAccumulator, which works
-  with the old API (for now). (Shai Erera) 
+  with the old API (for now). (Shai Erera)
 
 * LUCENE-4761: Facet packages reorganized. Should be easy to fix your import
   statements, if you use an IDE such as Eclipse. (Shai Erera)
@@ -9229,13 +9231,13 @@ Changes in backwards compatibility policy
 * LUCENE-4750: Convert DrillDown to DrillDownQuery, so you can initialize it
   and add drill-down categories to it. (Michael McCandless, Shai Erera)
 
-* LUCENE-4759: remove FacetRequest.SortBy; result categories are always 
+* LUCENE-4759: remove FacetRequest.SortBy; result categories are always
   sorted by value, while ties are broken by category ordinal. (Shai Erera)
 
 * LUCENE-4772: Facet associations moved to new FacetsAggregator API. You
   should override FacetsAccumulator and return the relevant aggregator,
   for aggregating the association values. (Shai Erera)
-  
+
 * LUCENE-4748: A FacetRequest on a non-existent field now returns an
   empty FacetResult instead of skipping it.  (Shai Erera, Mike McCandless)
 
@@ -9247,7 +9249,7 @@ Changes in backwards compatibility policy
 Optimizations
 
 * LUCENE-4687: BloomFilterPostingsFormat now lazily initializes delegate
-  TermsEnum only if needed to do a seek or get a DocsEnum. (Simon Willnauer) 
+  TermsEnum only if needed to do a seek or get a DocsEnum. (Simon Willnauer)
 
 * LUCENE-4677, LUCENE-4682: unpacked FSTs now use vInt to encode the node target,
   to reduce their size (Mike McCandless)
@@ -9264,7 +9266,7 @@ Optimizations
 
 * LUCENE-4715: CategoryListParams.getOrdinalPolicy now allows to return a
   different OrdinalPolicy per dimension, to better tune how you index
-  facets. Also added OrdinalPolicy.ALL_BUT_DIMENSION. 
+  facets. Also added OrdinalPolicy.ALL_BUT_DIMENSION.
   (Shai Erera, Michael McCandless)
 
 * LUCENE-4740: Don't track clones of MMapIndexInput if unmapping
@@ -9276,7 +9278,7 @@ Optimizations
 
 * LUCENE-3729: The default Lucene 4.2 codec now uses a more compact
   DocValuesFormat (Lucene42DocValuesFormat). Sorted values are stored in an
-  FST, Numerics and Ordinals use a number of strategies (delta-compression, 
+  FST, Numerics and Ordinals use a number of strategies (delta-compression,
   table-compression, etc), and memory addresses use MonotonicBlockPackedWriter.
   (Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
 
@@ -9289,7 +9291,7 @@ Optimizations
 
 New Features
 
-* LUCENE-4686: New specialized DGapVInt8IntEncoder for facets (now the 
+* LUCENE-4686: New specialized DGapVInt8IntEncoder for facets (now the
   default). (Shai Erera)
 
 * LUCENE-4703: Add simple PrintTaxonomyStats tool to see summary
@@ -9309,7 +9311,7 @@ New Features
   creation via the resulting factories using NewAnalyzerTask.  (Steve Rowe)
 
 * LUCENE-4728: Unknown and not explicitly mapped queries are now rewritten
-  against the highlighting IndexReader to obtain primitive queries before 
+  against the highlighting IndexReader to obtain primitive queries before
   discarding the query entirely. WeightedSpanTermExtractor now builds a
   MemoryIndex only once even if multiple fields are highlighted.
   (Simon Willnauer)
@@ -9318,15 +9320,15 @@ New Features
   support for Locale-sensitive sort and range queries for
   single-valued fields.  (Robert Muir)
 
-* LUCENE-4547: Added MonotonicBlockPacked(Reader/Writer), which provide 
-  efficient random access to large amounts of monotonically increasing 
+* LUCENE-4547: Added MonotonicBlockPacked(Reader/Writer), which provide
+  efficient random access to large amounts of monotonically increasing
   positive values (e.g. file offsets). Each block stores the minimum value
   and the average gap, and values are encoded as signed deviations from
   the expected value.  (Adrien Grand)
-  
+
 * LUCENE-4547: Added AppendingLongBuffer, an append-only buffer that packs
   signed long values in memory and provides an efficient iterator API.
-  (Adrien Grand) 
+  (Adrien Grand)
 
 * LUCENE-4540: It is now possible for a codec to represent norms with
   less than 8 bits per value. For performance reasons this is not done
@@ -9336,13 +9338,13 @@ New Features
 
 * LUCENE-4764: A new Facet42Codec and Facet42DocValuesFormat provide
   faster but more RAM-consuming facet performance.  (Shai Erera, Mike
-  McCandless) 
+  McCandless)
 
 * LUCENE-4769: Added OrdinalsCache and CachedOrdsCountingFacetsAggregator
   which uses the cache to obtain a document's ordinals. This aggregator
   is faster than others, however consumes much more RAM.
   (Michael McCandless, Shai Erera)
-  
+
 * LUCENE-4778: Add a getter for the delegate in RateLimitedDirectoryWrapper.
   (Mark Miller)
 
@@ -9355,7 +9357,7 @@ New Features
 * LUCENE-4748: Added DrillSideways utility class for computing both
   drill-down and drill-sideways counts for a DrillDownQuery.  (Mike
   McCandless)
- 
+
 API Changes
 
 * LUCENE-4709: FacetResultNode no longer has a residue field. (Shai Erera)
@@ -9370,13 +9372,13 @@ API Changes
 
 * LUCENE-4794: Spatial RecursivePrefixTreeFilter replaced by
   IntersectsPrefixTreeFilter and some extensible base classes. (David Smiley)
-  
+
 Bug Fixes
 
-* LUCENE-4705: Pass on FilterStrategy in FilteredQuery if the filtered query is 
+* LUCENE-4705: Pass on FilterStrategy in FilteredQuery if the filtered query is
   rewritten. (Simon Willnauer)
 
-* LUCENE-4712: MemoryIndex#normValues() throws NPE if field doesn't exist. 
+* LUCENE-4712: MemoryIndex#normValues() throws NPE if field doesn't exist.
   (Simon Willnauer, Ricky Pritchett)
 
 * LUCENE-4550: Shapes wider than 180 degrees would use too much accuracy for the
@@ -9398,13 +9400,13 @@ Bug Fixes
 * LUCENE-4739: Fixed bugs that prevented FSTs more than ~1.1GB from
   being saved and loaded (Adrien Grand, Mike McCandless)
 
-* LUCENE-4717: Fixed bug where Lucene40DocValuesFormat would sometimes write 
+* LUCENE-4717: Fixed bug where Lucene40DocValuesFormat would sometimes write
   an extra unused ordinal for sorted types. The bug is detected and corrected
   on-the-fly for old indexes.  (Robert Muir)
 
 * LUCENE-4547: Fixed bug where Lucene40DocValuesFormat was unable to encode
   segments that would exceed 2GB total data. This could happen in some surprising
-  cases, for example if you had an index with more than 260M documents and a 
+  cases, for example if you had an index with more than 260M documents and a
   VAR_INT field.  (Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
 
 * LUCENE-4775: Remove SegmentInfo.sizeInBytes() and make
@@ -9429,7 +9431,7 @@ Bug Fixes
   large performance impacts for many non-random or non-uniform
   term distributions.  (John Wang, yonik)
 
-* LUCENE-4798: PostingsHighlighter's formatter sometimes didn't highlight 
+* LUCENE-4798: PostingsHighlighter's formatter sometimes didn't highlight
   matched terms.  (Robert Muir)
 
 * LUCENE-4796, SOLR-4373: Fix concurrency issue in NamedSPILoader and
@@ -9447,11 +9449,11 @@ Documentation
   (Hayden Muhl via Adrien Grand)
 
 * LUCENE-4784, LUCENE-4785, LUCENE-4786: Fixed references to deprecated classes
-  SinkTokenizer, ValueSourceQuery and RangeQuery. (Hao Zhong via Adrien Grand) 
+  SinkTokenizer, ValueSourceQuery and RangeQuery. (Hao Zhong via Adrien Grand)
 
 Build
 
-* LUCENE-4654: Test duration statistics from multiple test runs should be 
+* LUCENE-4654: Test duration statistics from multiple test runs should be
   reused. (Dawid Weiss)
 
 * LUCENE-4636: Upgrade ivy to 2.3.0 (Shawn Heisey via Robert Muir)
@@ -9473,13 +9475,13 @@ Changes in backwards compatibility policy
   (Robert Muir)
 
 * LUCENE-4543: TFIDFSimilarity's index-time computeNorm is now final to
-  match the fact that its query-time norm usage requires a FIXED_8 encoding. 
+  match the fact that its query-time norm usage requires a FIXED_8 encoding.
   Override lengthNorm and/or encode/decodeNormValue to change the specifics,
   like Lucene 3.x. (Robert Muir)
 
 * LUCENE-3441: The facet module now supports NRT. As a result, the following
   changes were made:
-  - DirectoryTaxonomyReader has a new constructor which takes a 
+  - DirectoryTaxonomyReader has a new constructor which takes a
     DirectoryTaxonomyWriter. You should use that constructor in order to get
     the NRT support (or the old one for non-NRT).
   - TaxonomyReader.refresh() removed in exchange for TaxonomyReader.openIfChanged
@@ -9511,7 +9513,7 @@ Changes in backwards compatibility policy
   into one getParallelTaxonomyArrays(). You can obtain the 3 arrays that the
   previous two methods returned by calling parents(), children() or siblings()
   on the returned ParallelTaxonomyArrays. (Shai Erera)
-  
+
 * LUCENE-4585: Spatial PrefixTree based Strategies (either TermQuery or
   RecursivePrefix based) MAY want to re-index if used for point data. If a
   re-index is not done, then an indexed point is ~1/2 the smallest grid cell
@@ -9521,7 +9523,7 @@ Changes in backwards compatibility policy
 * LUCENE-4604: DefaultOrdinalPolicy removed in favor of OrdinalPolicy.ALL_PARENTS.
   Same for DefaultPathPolicy (now PathPolicy.ALL_CATEGORIES). In addition, you
   can use OrdinalPolicy.NO_PARENTS to never write any parent category ordinal
-  to the fulltree posting payload (but note that you need a special 
+  to the fulltree posting payload (but note that you need a special
   FacetsAccumulator - see javadocs). (Shai Erera)
 
 * LUCENE-4594: Spatial PrefixTreeStrategy no longer indexes center points of
@@ -9542,12 +9544,12 @@ Changes in backwards compatibility policy
 * LUCENE-4647: CategoryDocumentBuilder and EnhancementsDocumentBuilder are replaced
   by FacetFields and AssociationsFacetFields respectively. CategoryEnhancement and
   AssociationEnhancement were removed in favor of a simplified CategoryAssociation
-  interface, with CategoryIntAssociation and CategoryFloatAssociation 
+  interface, with CategoryIntAssociation and CategoryFloatAssociation
   implementations.
   NOTE: indexes that contain category enhancements/associations are not supported
   by the new code and should be recreated. (Shai Erera)
 
-* LUCENE-4659: Massive cleanup to CategoryPath API. Additionally, CategoryPath is 
+* LUCENE-4659: Massive cleanup to CategoryPath API. Additionally, CategoryPath is
   now immutable, so you don't need to clone() it. (Shai Erera)
 
 * LUCENE-4670: StoredFieldsWriter and TermVectorsWriter have new finish* callbacks
@@ -9561,7 +9563,7 @@ Changes in backwards compatibility policy
 * LUCENE-4683: CategoryListIterator and Aggregator are now per-segment. As such
   their implementations no longer take a top-level IndexReader in the constructor
   but rather implement a setNextReader. (Shai Erera)
-  
+
 New Features
 
 * LUCENE-4226: New experimental StoredFieldsFormat that compresses chunks of
@@ -9571,7 +9573,7 @@ New Features
   DocValues fields. (Adrien Grand)
 
 * LUCENE-4410: FilteredQuery now exposes a FilterStrategy that exposes
-  how filters are applied during query execution. (Simon Willnauer) 
+  how filters are applied during query execution. (Simon Willnauer)
 
 * LUCENE-4404: New ListOfOutputs (in lucene/misc) for FSTs wraps
   another Outputs implementation, allowing you to store more than one
@@ -9589,7 +9591,7 @@ New Features
 
 * LUCENE-4446: Lucene 4.1 has a new default index format (Lucene41Codec)
   that incorporates the previously experimental "Block" postings format
-  for better search performance. 
+  for better search performance.
   (Han Jiang, Adrien Grand, Robert Muir, Mike McCandless)
 
 * LUCENE-3846: New FuzzySuggester, like AnalyzingSuggester except it
@@ -9619,7 +9621,7 @@ New Features
   offsets from the postings lists to highlight documents. (Robert Muir)
 
 * LUCENE-4628: Added CommonTermsQuery that executes high-frequency terms
-  in a optional sub-query to prevent slow queries due to "common" terms 
+  in a optional sub-query to prevent slow queries due to "common" terms
   like stopwords. (Simon Willnauer)
 
 API Changes
@@ -9631,7 +9633,7 @@ API Changes
   to TokenStream.getTokenStreamWithOffsets, and return null on failure
   rather than throwing IllegalArgumentException.  (Alan Woodward)
 
-* LUCENE-4472: MergePolicy now accepts a MergeTrigger that provides 
+* LUCENE-4472: MergePolicy now accepts a MergeTrigger that provides
   information about the trigger of the merge ie. merge triggered due
   to a segment merge or a full flush etc. (Simon Willnauer)
 
@@ -9640,14 +9642,14 @@ API Changes
 
 * LUCENE-4520: ValueSource.getSortField no longer throws IOExceptions
   (Alan Woodward)
-  
+
 * LUCENE-4537: RateLimiter is now separated from FSDirectory and exposed via
   RateLimitingDirectoryWrapper. Any Directory can now be rate-limited.
-  (Simon Willnauer)  
+  (Simon Willnauer)
 
 * LUCENE-4591: CompressingStoredFields{Writer,Reader} now accept a segment
   suffix as a constructor parameter. (Renaud Delbru via Adrien Grand)
-  
+
 * LUCENE-4605: Added DocsEnum.FLAG_NONE which can be passed instead of 0 as
   the flag to .docs() and .docsAndPositions(). (Shai Erera)
 
@@ -9663,15 +9665,15 @@ API Changes
 
 * LUCENE-4684: Made DirectSpellChecker extendable.
   (Martijn van Groningen)
-  
+
 Bug Fixes
 
 * LUCENE-1822: BaseFragListBuilder hard-coded 6 char margin is too naive.
   (Alex Vigdor, Arcadius Ahouansou, Koji Sekiguchi)
 
-* LUCENE-4468: Fix rareish integer overflows in Lucene41 postings 
+* LUCENE-4468: Fix rareish integer overflows in Lucene41 postings
   format. (Robert Muir)
-  
+
 * LUCENE-4486: Add support for ConstantScoreQuery in Highlighter.
  (Simon Willnauer)
 
@@ -9685,7 +9687,7 @@ Bug Fixes
   romaji even for out-of-vocabulary kana cases (e.g. half-width forms).
   (Robert Muir)
 
-* LUCENE-4511: TermsFilter might return wrong results if a field is not 
+* LUCENE-4511: TermsFilter might return wrong results if a field is not
   indexed or doesn't exist in the index. (Simon Willnauer)
 
 * LUCENE-4521: IndexWriter.tryDeleteDocument could return true
@@ -9723,8 +9725,8 @@ Bug Fixes
 * LUCENE-4009: Improve TermsFilter.toString (Tim Costermans via Chris
   Male, Mike McCandless)
 
-* LUCENE-4588: Benchmark's EnwikiContentSource was discarding last wiki 
-  document and had leaking threads in 'forever' mode. (Doron Cohen)   
+* LUCENE-4588: Benchmark's EnwikiContentSource was discarding last wiki
+  document and had leaking threads in 'forever' mode. (Doron Cohen)
 
 * LUCENE-4585: Spatial RecursivePrefixTreeFilter had some bugs that only
   occurred when shapes were indexed.  In what appears to be rare circumstances,
@@ -9732,7 +9734,7 @@ Bug Fixes
   In addition, it wasn't possible to index a shape representing the entire
   globe.
 
-* LUCENE-4595: EnwikiContentSource had a thread safety problem (NPE) in 
+* LUCENE-4595: EnwikiContentSource had a thread safety problem (NPE) in
   'forever' mode (Doron Cohen)
 
 * LUCENE-4587: fix WordBreakSpellChecker to not throw AIOOBE when presented
@@ -9779,13 +9781,13 @@ Bug Fixes
 
 * LUCENE-4465: Let ConstantScoreQuery's Scorer return its child scorer.
   (selckin via Uwe Schindler)
-    
+
 Changes in Runtime Behavior
-  
+
 * LUCENE-4586: Change default ResultMode of FacetRequest to PER_NODE_IN_TREE.
   This only affects requests with depth>1. If you execute such requests and
   rely on the facet results being returned flat (i.e. no hierarchy), you should
-  set the ResultMode to GLOBAL_FLAT. (Shai Erera, Gilad Barkai) 
+  set the ResultMode to GLOBAL_FLAT. (Shai Erera, Gilad Barkai)
 
 * LUCENE-1822: Improves the text window selection by recalculating the starting margin
   once all phrases in the fragment have been identified in FastVectorHighlighter. This
@@ -9793,7 +9795,7 @@ Changes in Runtime Behavior
   instead of 6 characters from the beginning. This way one can also guarantee that
   the entirety of short texts are represented in a fragment by specifying a large
   enough fragCharSize.
-  
+
 Optimizations
 
 * LUCENE-2221: oal.util.BitUtil was modified to use Long.bitCount and
@@ -9811,7 +9813,7 @@ Optimizations
 * LUCENE-4512: Additional memory savings for CompressingStoredFieldsFormat.
   (Adrien Grand, Robert Muir)
 
-* LUCENE-4443: Lucene41PostingsFormat no longer writes unnecessary offsets 
+* LUCENE-4443: Lucene41PostingsFormat no longer writes unnecessary offsets
   into the skipdata. (Robert Muir)
 
 * LUCENE-4459: Improve WeakIdentityMap.keyIterator() to remove GCed keys
@@ -9831,23 +9833,23 @@ Optimizations
   the postings data (via flags to TermsEnum.docs/docsAndPositions) to use
   ForUtil.skipBlock.  (Robert Muir)
 
-* LUCENE-4497: Don't write PosVIntCount to the positions file in 
+* LUCENE-4497: Don't write PosVIntCount to the positions file in
   Lucene41PostingsFormat, as it's always totalTermFreq % BLOCK_SIZE. (Robert Muir)
 
-* LUCENE-4498: In Lucene41PostingsFormat, when a term appears in only one document, 
-  Instead of writing a file pointer to a VIntBlock containing the doc id, just 
+* LUCENE-4498: In Lucene41PostingsFormat, when a term appears in only one document,
+  Instead of writing a file pointer to a VIntBlock containing the doc id, just
   write the doc id.  (Mike McCandless, Robert Muir)
 
-* LUCENE-4515: MemoryIndex now uses Byte/IntBlockPool internally to hold terms and 
-  posting lists. All index data is represented as consecutive byte/int arrays to 
-  reduce GC cost and memory overhead. (Simon Willnauer) 
+* LUCENE-4515: MemoryIndex now uses Byte/IntBlockPool internally to hold terms and
+  posting lists. All index data is represented as consecutive byte/int arrays to
+  reduce GC cost and memory overhead. (Simon Willnauer)
 
-* LUCENE-4538: DocValues now caches direct sources in a ThreadLocal exposed via SourceCache. 
+* LUCENE-4538: DocValues now caches direct sources in a ThreadLocal exposed via SourceCache.
   Users of this API can now simply obtain an instance via DocValues#getDirectSource per thread.
   (Simon Willnauer)
-  
+
 * LUCENE-4580: DrillDown.query variants return a ConstantScoreQuery with boost set to 0.0f
-  so that documents scores are not affected by running a drill-down query. (Shai Erera)  
+  so that documents scores are not affected by running a drill-down query. (Shai Erera)
 
 * LUCENE-4598: PayloadIterator no longer uses top-level IndexReader to iterate on the
   posting's payload. (Shai Erera, Michael McCandless)
@@ -9855,7 +9857,7 @@ Optimizations
 * LUCENE-4661: Drop default maxThreadCount to 1 and maxMergeCount to 2
   in ConcurrentMergeScheduler, for faster merge performance on
   spinning-magnet drives (Mike McCandless)
-  
+
 Documentation
 
 * LUCENE-4483: Refer to BytesRef.deepCopyOf in Term's constructor that takes BytesRef.
@@ -9870,10 +9872,10 @@ Build
   JVM PIDs on heartbeat from hung tests (Dawid Weiss)
 
 * Upgrade randomized testing to version 2.0.4: avoid hangs on shutdown
-  hooks hanging forever by calling Runtime.halt() in addition to 
+  hooks hanging forever by calling Runtime.halt() in addition to
   Runtime.exit() after a short delay to allow graceful shutdown (Dawid Weiss)
 
-* LUCENE-4451: Memory leak per unique thread caused by 
+* LUCENE-4451: Memory leak per unique thread caused by
   RandomizedContext.contexts static map. Upgrade randomized testing
   to version 2.0.2 (Mike McCandless, Dawid Weiss)
 
@@ -9883,8 +9885,8 @@ Build
 
 * LUCENE-4601: Fix ivy availability check to use typefound, so it works
   if called from another build file.  (Ryan Ernst via Robert Muir)
-  
-  
+
+
 ======================= Lucene 4.0.0 =======================
 
 Changes in backwards compatibility policy
@@ -9898,7 +9900,7 @@ Changes in backwards compatibility policy
 New Features
 
 * LUCENE-1888: Added the option to store payloads in the term
-  vectors (IndexableFieldType.storeTermVectorPayloads()). Note 
+  vectors (IndexableFieldType.storeTermVectorPayloads()). Note
   that you must store term vector positions to store payloads.
   (Robert Muir)
 
@@ -9910,7 +9912,7 @@ New Features
 * LUCENE-4323: Added support for an absolute maximum CFS segment size
   (in MiB) to LogMergePolicy and TieredMergePolicy.
   (Alexey Lef via Uwe Schindler)
-  
+
 * LUCENE-4339: Allow deletes against 3.x segments for easier upgrading.
   Lucene3x Codec is still otherwise read-only, you should not set it
   as the default Codec on IndexWriter, because it cannot write new segments.
@@ -9918,18 +9920,18 @@ New Features
 
 * SOLR-3441: ElisionFilterFactory is now MultiTermAware
   (Jack Krupansky via hossman)
-  
+
 API Changes
 
 * LUCENE-4391, LUCENE-4440: All methods of Lucene40Codec but
-  getPostingsFormatForField are now final. To reuse functionality 
+  getPostingsFormatForField are now final. To reuse functionality
   of Lucene40, you should extend FilterCodec and delegate to Lucene40
   instead of extending Lucene40Codec.  (Adrien Grand, Shai Erera,
   Robert Muir, Uwe Schindler)
 
 * LUCENE-4299: Added Terms.hasPositions() and Terms.hasOffsets().
   Previously you had no real way to know that a term vector field
-  had positions or offsets, since this can be configured on a 
+  had positions or offsets, since this can be configured on a
   per-field-per-document basis. (Robert Muir)
 
 * Removed DocsAndPositionsEnum.hasPayload() and simplified the
@@ -9957,21 +9959,21 @@ API Changes
   leaves() (LUCENE-4152), which lists AtomicReaderContexts including
   the doc base of each leaf.  (Uwe Schindler, Robert Muir)
 
-* LUCENE-4307: Renamed IndexReader.getTopReaderContext to 
+* LUCENE-4307: Renamed IndexReader.getTopReaderContext to
   IndexReader.getContext.  (Robert Muir)
 
 * LUCENE-4316: Deprecate Fields.getUniqueTermCount and remove it from
   AtomicReader. If you really want the unique term count across all
   fields, just sum up Terms.size() across those fields. This method
-  only exists so that this statistic can be accessed for Lucene 3.x 
+  only exists so that this statistic can be accessed for Lucene 3.x
   segments, which don't support Terms.size().  (Uwe Schindler, Robert Muir)
 
 * LUCENE-4321: Change CharFilter to extend Reader directly, as FilterReader
   overdelegates (read(), read(char[], int, int), skip, etc). This made it
   hard to implement CharFilters that were correct. Instead only close() is
   delegated by default: read(char[], int, int) and correct(int) are abstract
-  so that it's obvious which methods you should implement.  The protected 
-  inner Reader is 'input' like CharFilter in the 3.x series, instead of 'in'.  
+  so that it's obvious which methods you should implement.  The protected
+  inner Reader is 'input' like CharFilter in the 3.x series, instead of 'in'.
   (Dawid Weiss, Uwe Schindler, Robert Muir)
 
 * LUCENE-3309: The expert FieldSelector API, used to load only certain
@@ -9982,7 +9984,7 @@ API Changes
   not be overridden by subclasses: per-stream initialization should happen
   in reset().  (Robert Muir)
 
-* LUCENE-4377: Remove IndexInput.copyBytes(IndexOutput, long). 
+* LUCENE-4377: Remove IndexInput.copyBytes(IndexOutput, long).
   Use DataOutput.copyBytes(DataInput, long) instead.
   (Mike McCandless, Robert Muir)
 
@@ -10016,8 +10018,8 @@ Bug Fixes
   query would be scored differently.  (Robert Muir)
 
 * Don't allow negatives in the positions file. If you have an index
-  from 2.4.0 or earlier with such negative positions, and you already 
-  upgraded to 3.x, then to Lucene 4.0-ALPHA or -BETA, you should run 
+  from 2.4.0 or earlier with such negative positions, and you already
+  upgraded to 3.x, then to Lucene 4.0-ALPHA or -BETA, you should run
   CheckIndex. If it fails, then you need to upgrade again to 4.0  (Robert Muir)
 
 * LUCENE-4303: PhoneticFilterFactory and SnowballPorterFilterFactory load their
@@ -10068,7 +10070,7 @@ Bug Fixes
   many cases where IndexWriter could leave leftover files (on
   exception in various places, on reuse of a segment name after crash
   and recovery.  (Uwe Schindler, Robert Muir, Mike McCandless)
-  
+
 Optimizations
 
 * LUCENE-4322: Decrease lucene-core JAR size. The core JAR size had increased a
@@ -10083,7 +10085,7 @@ Optimizations
 
 * LUCENE-4364: Optimize MMapDirectory to not make a mapping per-cfs-slice,
   instead one map per .cfs file. This reduces the total number of maps.
-  Additionally factor out a (package-private) generic 
+  Additionally factor out a (package-private) generic
   ByteBufferIndexInput from MMapDirectory.  (Uwe Schindler, Robert Muir)
 
 Build
@@ -10096,10 +10098,10 @@ Build
 * LUCENE-4252: Detect/Fail tests when they leak RAM in static fields
   (Robert Muir, Dawid Weiss)
 
-* LUCENE-4360: Support running the same test suite multiple times in 
+* LUCENE-4360: Support running the same test suite multiple times in
   parallel (Dawid Weiss)
 
-* LUCENE-3985: Upgrade to randomizedtesting 2.0.0. Added support for 
+* LUCENE-3985: Upgrade to randomizedtesting 2.0.0. Added support for
   thread leak detection. Added support for suite timeouts. (Dawid Weiss)
 
 * LUCENE-4354: Corrected maven dependencies to be consistent with
@@ -10121,10 +10123,10 @@ New features
 * LUCENE-4249: Changed the explanation of the PayloadTermWeight to use the
   underlying PayloadFunction's explanation as the explanation
   for the payload score. (Scott Smerchek via Robert Muir)
-  
+
 * LUCENE-4069: Added BloomFilteringPostingsFormat for use with low-frequency terms
-  such as primary keys (Mark Harwood, Mike McCandless) 
-  
+  such as primary keys (Mark Harwood, Mike McCandless)
+
 * LUCENE-4201: Added JapaneseIterationMarkCharFilter to normalize Japanese
   iteration marks. (Robert Muir, Christian Moen)
 
@@ -10154,9 +10156,9 @@ New features
   int docID), to attempt deletion by docID as long as the provided
   reader is an NRT reader, and the segment has not yet been merged
   away (Mike McCandless).
-  
+
 * LUCENE-4286: Added option to CJKBigramFilter to always also output
-  unigrams. This can be used for a unigram+bigram approach, or at 
+  unigrams. This can be used for a unigram+bigram approach, or at
   index-time only for better support of short queries.
   (Tom Burton-West, Robert Muir)
 
@@ -10207,9 +10209,9 @@ API Changes
   instead of the previous boolean needsFlags; consistent with the changes
   for DocsAndPositionsEnum in LUCENE-4230. Currently the only flag
   is DocsEnum.FLAG_FREQS. (Robert Muir, Mike McCandless)
-  
+
 * LUCENE-3616: TextField(String, Reader, Store) was reduced to TextField(String, Reader),
-  as the Store parameter didn't make sense: if you supplied Store.YES, you would only 
+  as the Store parameter didn't make sense: if you supplied Store.YES, you would only
   receive an exception anyway. (Robert Muir)
 
 Optimizations
@@ -10224,21 +10226,21 @@ Optimizations
   (Uwe Schindler)
 
 * LUCENE-4279: Regenerated snowball Stemmers from snowball r554,
-  making them substantially more lightweight. Behavior is unchanged. 
+  making them substantially more lightweight. Behavior is unchanged.
   (Robert Muir)
 
-* LUCENE-4291: Reduced internal buffer size for Jflex-based tokenizers 
-  such as StandardTokenizer from 32kb to 8kb.  
+* LUCENE-4291: Reduced internal buffer size for Jflex-based tokenizers
+  such as StandardTokenizer from 32kb to 8kb.
   (Raintung Li, Steven Rowe, Robert Muir)
 
 Bug Fixes
 
-* LUCENE-4109: BooleanQueries are not parsed correctly with the 
+* LUCENE-4109: BooleanQueries are not parsed correctly with the
   flexible query parser. (Karsten Rauch via Robert Muir)
 
 * LUCENE-4176: Fix AnalyzingQueryParser to analyze range endpoints as bytes,
   so that it works correctly with Analyzers that produce binary non-UTF-8 terms
-  such as CollationAnalyzer. (Nattapong Sirilappanich via Robert Muir) 
+  such as CollationAnalyzer. (Nattapong Sirilappanich via Robert Muir)
 
 * LUCENE-4209: Fix FSTCompletionLookup to close its sorter, so that it won't
   leave temp files behind in /tmp. Fix SortedTermFreqIteratorWrapper to not
@@ -10253,15 +10255,15 @@ Bug Fixes
 
 * LUCENE-3505: Fix bug (Lucene 4.0alpha only) where boolean conjunctions
   were sometimes scored incorrectly. Conjunctions of only termqueries where
-  at least one term omitted term frequencies (IndexOptions.DOCS_ONLY) would 
+  at least one term omitted term frequencies (IndexOptions.DOCS_ONLY) would
   be scored as if all terms omitted term frequencies.  (Robert Muir)
 
-* LUCENE-2686, LUCENE-3505: Fixed BooleanQuery scorers to return correct 
-  freq().  Added support for scorer navigation API (Scorer.getChildren) to 
-  all queries.  Made Scorer.freq() abstract. 
+* LUCENE-2686, LUCENE-3505: Fixed BooleanQuery scorers to return correct
+  freq().  Added support for scorer navigation API (Scorer.getChildren) to
+  all queries.  Made Scorer.freq() abstract.
   (Koji Sekiguchi, Mike McCandless, Robert Muir)
 
-* LUCENE-4234: Exception when FacetsCollector is used with ScoreFacetRequest, 
+* LUCENE-4234: Exception when FacetsCollector is used with ScoreFacetRequest,
   and the number of matching documents is too large. (Gilad Barkai via Shai Erera)
 
 * LUCENE-4245: Make IndexWriter#close() and MergeScheduler#close()
@@ -10272,9 +10274,9 @@ Bug Fixes
   IndexWriter to only delete files matching this pattern from an index
   directory, to reduce risk when the wrong index path is accidentally
   passed to IndexWriter (Robert Muir, Mike McCandless)
-  
+
 * LUCENE-4277: Fix IndexWriter deadlock during rollback if flushable DWPT
-  instance are already checked out and queued up but not yet flushed. 
+  instance are already checked out and queued up but not yet flushed.
   (Simon Willnauer)
 
 * LUCENE-4282: Automaton FuzzyQuery didn't always deliver all results.
@@ -10287,8 +10289,8 @@ Changes in Runtime Behavior
 
 * LUCENE-4109: Enable position increments in the flexible queryparser by default.
   (Karsten Rauch via Robert Muir)
-  
-* LUCENE-3616: Field throws exception if you try to set a boost on an 
+
+* LUCENE-3616: Field throws exception if you try to set a boost on an
   unindexed field or one that omits norms. (Robert Muir)
 
 Build
@@ -10296,7 +10298,7 @@ Build
 * LUCENE-4094: Support overriding file.encoding on forked test JVMs
   (force via -Drandomized.file.encoding=XXX). (Dawid Weiss)
 
-* LUCENE-4189: Test output should include timestamps (start/end for each 
+* LUCENE-4189: Test output should include timestamps (start/end for each
   test/ suite). Added -Dtests.timestamps=[off by default]. (Dawid Weiss)
 
 * LUCENE-4110: Report long periods of forked jvm inactivity (hung tests/ suites).
@@ -10310,7 +10312,7 @@ Build
   specify -Dtests.failfast=true to skip all tests after the first failure.
   (Dawid Weiss)
 
-* LUCENE-4115: JAR resolution/ cleanup should be done automatically for ant 
+* LUCENE-4115: JAR resolution/ cleanup should be done automatically for ant
   clean/ eclipse/ resolve (Dawid Weiss)
 
 * LUCENE-4199, LUCENE-4202, LUCENE-4206: Add a new target "check-forbidden-apis"
@@ -10318,19 +10320,19 @@ Build
   charset, default locale, or default timezone and fail build if violations
   found. This ensures, that Lucene / Solr is independent on local configuration
   options.  (Uwe Schindler, Robert Muir, Dawid Weiss)
-  
+
 * LUCENE-4217: Add the possibility to run tests with Atlassian Clover
   loaded from IVY. A development License solely for Apache code was added in
   the tools/ folder, but is not included in releases.  (Uwe Schindler)
 
 Documentation
 
-* LUCENE-4195: Added package documentation and examples for 
+* LUCENE-4195: Added package documentation and examples for
   org.apache.lucene.codecs (Alan Woodward via Robert Muir)
 
 ======================= Lucene 4.0.0-ALPHA =======================
 
-More information about this release, including any errata related to the 
+More information about this release, including any errata related to the
 release notes, upgrade instructions, or other changes may be found online at:
    https://wiki.apache.org/lucene-java/Lucene4.0
 
@@ -10364,7 +10366,7 @@ Changes in backwards compatibility policy
     now specify the skipDocs explicitly (typically this will be the
     deleted docs, but in general you can provide any Bits).
 
-  - The term vectors APIs (TermFreqVector, TermPositionVector, 
+  - The term vectors APIs (TermFreqVector, TermPositionVector,
     TermVectorMapper) have been removed in favor of the above
     flexible indexing APIs, presenting a single-document inverted
     index of the document from the term vectors.
@@ -10401,13 +10403,13 @@ Changes in backwards compatibility policy
     with upper and lower bound equal and included. TermQueries do not
     score correct, so the constant score mode of NRQ is the only
     correct way to handle single value queries.
-  
+
   - NumericTokenStream now works directly on byte[] terms. If you
     plug a TokenFilter on top of this stream, you will likely get
     an IllegalArgumentException, because the NTS does not support
     TermAttribute/CharTermAttribute. If you want to further filter
     or attach Payloads to NTS, use the new NumericTermAttribute.
-    
+
   (Mike McCandless, Robert Muir, Uwe Schindler, Mark Miller, Michael Busch)
 
 * LUCENE-2858, LUCENE-3733: IndexReader was refactored into abstract
@@ -10425,7 +10427,7 @@ Changes in backwards compatibility policy
 
 * LUCENE-2265: FuzzyQuery and WildcardQuery now operate on Unicode codepoints,
   not unicode code units. For example, a Wildcard "?" represents any unicode
-  character. Furthermore, the rest of the automaton package and RegexpQuery use 
+  character. Furthermore, the rest of the automaton package and RegexpQuery use
   true Unicode codepoint representation.  (Robert Muir, Mike McCandless)
 
 * LUCENE-2380: The String-based FieldCache methods (getStrings,
@@ -10434,32 +10436,32 @@ Changes in backwards compatibility policy
   FieldDoc.fields) when sorting by SortField.STRING or
   SortField.STRING_VAL are now BytesRef instances.  See MIGRATE.txt
   for more details. (yonik, Mike McCandless)
- 
-* LUCENE-2480: Though not a change in backwards compatibility policy, pre-3.0 
+
+* LUCENE-2480: Though not a change in backwards compatibility policy, pre-3.0
   indexes are no longer supported. You should upgrade to 3.x first, then run
   optimize(), or reindex. (Shai Erera, Earwin Burrfoot)
-  
+
 * LUCENE-2484: Removed deprecated TermAttribute. Use CharTermAttribute
   and TermToBytesRefAttribute instead.  (Uwe Schindler)
 
 * LUCENE-2600: Remove IndexReader.isDeleted in favor of
   AtomicReader.getDeletedDocs().  (Mike McCandless)
 
-* LUCENE-2667: FuzzyQuery's defaults have changed for more performant 
+* LUCENE-2667: FuzzyQuery's defaults have changed for more performant
   behavior: the minimum similarity is 2 edit distances from the word,
   and the priority queue size is 50. To support this, FuzzyQuery now allows
   specifying unscaled edit distances (foobar~2). If your application depends
   upon the old defaults of 0.5 (scaled) minimum similarity and Integer.MAX_VALUE
-  priority queue size, you can use FuzzyQuery(Term, float, int, int) to specify 
+  priority queue size, you can use FuzzyQuery(Term, float, int, int) to specify
   those explicitly.
-  
+
 * LUCENE-2674: MultiTermQuery.TermCollector.collect now accepts the
   TermsEnum as well.  (Robert Muir, Mike McCandless)
 
-* LUCENE-588: WildcardQuery and QueryParser now allows escaping with 
+* LUCENE-588: WildcardQuery and QueryParser now allows escaping with
   the '\' character. Previously this was impossible (you could not escape */?,
   for example).  If your code somehow depends on the old behavior, you will
-  need to change it (e.g. using "\\" to escape '\' itself).  
+  need to change it (e.g. using "\\" to escape '\' itself).
   (Sunil Kamath, Terry Yang via Robert Muir)
 
 * LUCENE-2837: Collapsed Searcher, Searchable into IndexSearcher;
@@ -10467,21 +10469,21 @@ Changes in backwards compatibility policy
   ParallelMultiSearcher into IndexSearcher as an optional
   ExecutorServiced passed to its ctor.  (Mike McCandless)
 
-* LUCENE-2908, LUCENE-4037: Removed serialization code from lucene classes. 
-  It is recommended that you serialize user search needs at a higher level 
+* LUCENE-2908, LUCENE-4037: Removed serialization code from lucene classes.
+  It is recommended that you serialize user search needs at a higher level
   in your application.
   (Robert Muir, Benson Margulies)
 
 * LUCENE-2831: Changed Weight#scorer, Weight#explain & Filter#getDocIdSet to
   operate on a AtomicReaderContext instead of directly on IndexReader to enable
   searches to be aware of IndexSearcher's context. (Simon Willnauer)
-  
+
 * LUCENE-2839: Scorer#score(Collector,int,int) is now public because it is
   called from other classes and part of public API. (Uwe Schindler)
-  
+
 * LUCENE-2865: Weight#scorer(AtomicReaderContext, boolean, boolean) now accepts
   a ScorerContext struct instead of booleans.(Simon Willnauer)
-  
+
 * LUCENE-2882: Cut over SpanQuery#getSpans to AtomicReaderContext to enforce
   per segment semantics on SpanQuery & Spans. (Simon Willnauer)
 
@@ -10500,31 +10502,31 @@ Changes in backwards compatibility policy
   interned.  (Mike McCandless)
 
 * LUCENE-2883: The contents of o.a.l.search.function has been consolidated into
-  the queries module and can be found at o.a.l.queries.function.  See 
+  the queries module and can be found at o.a.l.queries.function.  See
   MIGRATE.txt for more information (Chris Male)
 
-* LUCENE-2392, LUCENE-3299: Decoupled vector space scoring from 
-  Query/Weight/Scorer. If you extended Similarity directly before, you should 
-  extend TFIDFSimilarity instead.  Similarity is now a lower-level API to 
+* LUCENE-2392, LUCENE-3299: Decoupled vector space scoring from
+  Query/Weight/Scorer. If you extended Similarity directly before, you should
+  extend TFIDFSimilarity instead.  Similarity is now a lower-level API to
   implement other scoring algorithms.  See MIGRATE.txt for more details.
   (David Nemeskey, Simon Willnauer, Mike McCandless, Robert Muir)
 
 * LUCENE-3330: The expert visitor API in Scorer has been simplified and
-  extended to support arbitrary relationships. To navigate to a scorer's 
+  extended to support arbitrary relationships. To navigate to a scorer's
   children, call Scorer.getChildren().  (Robert Muir)
 
 * LUCENE-2308: Field is now instantiated with an instance of IndexableFieldType,
-  of which there is a core implementation FieldType.  Most properties 
-  describing a Field have been moved to IndexableFieldType.  See MIGRATE.txt 
+  of which there is a core implementation FieldType.  Most properties
+  describing a Field have been moved to IndexableFieldType.  See MIGRATE.txt
   for more details.  (Nikola Tankovic, Mike McCandless, Chris Male)
 
-* LUCENE-3396: ReusableAnalyzerBase.TokenStreamComponents.reset(Reader) now 
-  returns void instead of boolean.  If a Component cannot be reset, it should 
+* LUCENE-3396: ReusableAnalyzerBase.TokenStreamComponents.reset(Reader) now
+  returns void instead of boolean.  If a Component cannot be reset, it should
   throw an Exception.  (Chris Male)
 
 * LUCENE-3396: ReusableAnalyzerBase has been renamed to Analyzer.  All Analyzer
   implementations must now use Analyzer.TokenStreamComponents, rather than
-  overriding .tokenStream() and .reusableTokenStream() (which are now final). 
+  overriding .tokenStream() and .reusableTokenStream() (which are now final).
   (Chris Male)
 
 * LUCENE-3346: Analyzer.reusableTokenStream() has been renamed to tokenStream()
@@ -10545,10 +10547,10 @@ Changes in backwards compatibility policy
 * LUCENE-3559: The methods "docFreq" and "maxDoc" on IndexSearcher were removed,
   as these are no longer used by the scoring system. See MIGRATE.txt for more
   details.  (Robert Muir)
-  
-* LUCENE-3533: Removed SpanFilters, they created large lists of objects and 
+
+* LUCENE-3533: Removed SpanFilters, they created large lists of objects and
   did not scale. (Robert Muir)
-  
+
 * LUCENE-3606: IndexReader and subclasses were made read-only. It is no longer
   possible to delete or undelete documents using IndexReader; you have to use
   IndexWriter now. As deleting by internal Lucene docID is no longer possible,
@@ -10573,11 +10575,11 @@ Changes in backwards compatibility policy
   FieldInfo.IndexOption: DOCS_AND_POSITIONS_AND_OFFSETS.  (Robert
   Muir, Mike McCandless)
 
-* LUCENE-2858, LUCENE-3770: FilterIndexReader was renamed to 
-  FilterAtomicReader and now extends AtomicReader. If you want to filter 
-  composite readers like DirectoryReader or MultiReader, filter their 
-  atomic leaves and build a new CompositeReader (e.g. MultiReader) around 
-  them. (Uwe Schindler, Robert Muir) 
+* LUCENE-2858, LUCENE-3770: FilterIndexReader was renamed to
+  FilterAtomicReader and now extends AtomicReader. If you want to filter
+  composite readers like DirectoryReader or MultiReader, filter their
+  atomic leaves and build a new CompositeReader (e.g. MultiReader) around
+  them. (Uwe Schindler, Robert Muir)
 
 * LUCENE-3736: ParallelReader was split into ParallelAtomicReader
   and ParallelCompositeReader. Lucene 3.x's ParallelReader is now
@@ -10613,17 +10615,17 @@ Changes in Runtime Behavior
 
 * LUCENE-2846: omitNorms now behaves like omitTermFrequencyAndPositions, if you
   omitNorms(true) for field "a" for 1000 documents, but then add a document with
-  omitNorms(false) for field "a", all documents for field "a" will have no 
-  norms.  Previously, Lucene would fill the first 1000 documents with 
+  omitNorms(false) for field "a", all documents for field "a" will have no
+  norms.  Previously, Lucene would fill the first 1000 documents with
   "fake norms" from Similarity.getDefault(). (Robert Muir, Mike McCandless)
 
 * LUCENE-2846: When some documents contain field "a", and others do not, the
-  documents that don't have the field get a norm byte value of 0. Previously, 
-  Lucene would populate "fake norms" with Similarity.getDefault() for these 
+  documents that don't have the field get a norm byte value of 0. Previously,
+  Lucene would populate "fake norms" with Similarity.getDefault() for these
   documents.  (Robert Muir, Mike McCandless)
-  
-* LUCENE-2720: IndexWriter throws IndexFormatTooOldException on open, rather 
-  than later when e.g. a merge starts. 
+
+* LUCENE-2720: IndexWriter throws IndexFormatTooOldException on open, rather
+  than later when e.g. a merge starts.
   (Shai Erera, Mike McCandless, Uwe Schindler)
 
 * LUCENE-2881: FieldInfos is now tracked per segment.  Before it was tracked
@@ -10632,8 +10634,8 @@ Changes in Runtime Behavior
   globally across IndexWriter sessions and persisted into a _X.fnx file on
   successful commit. The corresponding file format changes are backwards-
   compatible. (Michael Busch, Simon Willnauer)
-  
-* LUCENE-2956, LUCENE-2573, LUCENE-2324, LUCENE-2555: Changes from 
+
+* LUCENE-2956, LUCENE-2573, LUCENE-2324, LUCENE-2555: Changes from
   DocumentsWriterPerThread:
 
   - IndexWriter now uses a DocumentsWriter per thread when indexing documents.
@@ -10647,8 +10649,8 @@ Changes in Runtime Behavior
     indexing may continue concurrently with flushing.  The selected
     DWPT flushes all its RAM resident documents do disk.  Note: Segment flushes
     don't flush all RAM resident documents but only the documents private to
-    the DWPT selected for flushing. 
-  
+    the DWPT selected for flushing.
+
   - Flushing is now controlled by FlushPolicy that is called for every add,
     update or delete on IndexWriter. By default DWPTs are flushed either on
     maxBufferedDocs per DWPT or the global active used memory. Once the active
@@ -10656,7 +10658,7 @@ Changes in Runtime Behavior
     flushing and the memory used by this DWPT is subtracted from the active
     memory and added to a flushing memory pool, which can lead to temporarily
     higher memory usage due to ongoing indexing.
-    
+
   - IndexWriter now can utilize ramBufferSize > 2048 MB. Each DWPT can address
     up to 2048 MB memory such that the ramBufferSize is now bounded by the max
     number of DWPT available in the used DocumentsWriterPerThreadPool.
@@ -10664,37 +10666,37 @@ Changes in Runtime Behavior
     the application can use all available DWPTs. To prevent a DWPT from
     exhausting its address space IndexWriter will forcefully flush a DWPT if its
     hard memory limit is exceeded. The RAMPerThreadHardLimitMB can be controlled
-    via IndexWriterConfig and defaults to 1945 MB. 
+    via IndexWriterConfig and defaults to 1945 MB.
     Since IndexWriter flushes DWPT concurrently not all memory is released
     immediately. Applications should still use a ramBufferSize significantly
     lower than the JVMs available heap memory since under high load multiple
     flushing DWPT can consume substantial transient memory when IO performance
     is slow relative to indexing rate.
-    
+
   - IndexWriter#commit now doesn't block concurrent indexing while flushing all
     'currently' RAM resident documents to disk. Yet, flushes that occur while a
     a full flush is running are queued and will happen after all DWPT involved
     in the full flush are done flushing. Applications using multiple threads
     during indexing and trigger a full flush (eg call commit() or open a new
     NRT reader) can use significantly more transient memory.
-    
+
   - IndexWriter#addDocument and IndexWriter.updateDocument can block indexing
     threads if the number of active + number of flushing DWPT exceed a
     safety limit. By default this happens if 2 * max number available thread
     states (DWPTPool) is exceeded. This safety limit prevents applications from
     exhausting their available memory if flushing can't keep up with
-    concurrently indexing threads.  
-    
+    concurrently indexing threads.
+
   - IndexWriter only applies and flushes deletes if the maxBufferedDelTerms
     limit is reached during indexing. No segment flushes will be triggered
     due to this setting.
-    
+
   - IndexWriter#flush(boolean, boolean) doesn't synchronized on IndexWriter
     anymore. A dedicated flushLock has been introduced to prevent multiple full-
-    flushes happening concurrently. 
-    
-  - DocumentsWriter doesn't write shared doc stores anymore. 
-  
+    flushes happening concurrently.
+
+  - DocumentsWriter doesn't write shared doc stores anymore.
+
   (Mike McCandless, Michael Busch, Simon Willnauer)
 
 * LUCENE-3309: Stored fields no longer record whether they were
@@ -10709,7 +10711,7 @@ Changes in Runtime Behavior
   (Mike McCandless)
 
 * LUCENE-2621: Removed contrib/instantiated.  (Robert Muir)
- 
+
 * LUCENE-1768: StandardQueryTreeBuilder no longer uses RangeQueryNodeBuilder
   for RangeQueryNodes, since theses two classes were removed;
   TermRangeQueryNodeProcessor now creates TermRangeQueryNode,
@@ -10749,13 +10751,13 @@ API Changes
 
 * LUCENE-2566: QueryParser: Unary operators +,-,! will not be treated as
   operators if they are followed by whitespace. (yonik)
-  
+
 * LUCENE-2831: Weight#scorer, Weight#explain, Filter#getDocIdSet,
   Collector#setNextReader & FieldComparator#setNextReader now expect an
   AtomicReaderContext instead of an IndexReader. (Simon Willnauer)
 
-* LUCENE-2892: Add QueryParser.newFieldQuery (called by getFieldQuery by 
-  default) which takes Analyzer as a parameter, for easier customization by 
+* LUCENE-2892: Add QueryParser.newFieldQuery (called by getFieldQuery by
+  default) which takes Analyzer as a parameter, for easier customization by
   subclasses.  (Robert Muir)
 
 * LUCENE-2953: In addition to changes in 3.x, PriorityQueue#initialize(int)
@@ -10786,7 +10788,7 @@ API Changes
   (Yonik Seeley, Andrzej Bialecki, Robert Muir)
 
 * LUCENE-3761: Generalize SearcherManager into an abstract ReferenceManager.
-  SearcherManager remains a concrete class, but due to the refactoring, the 
+  SearcherManager remains a concrete class, but due to the refactoring, the
   method maybeReopen has been deprecated in favor of maybeRefresh().
   (Shai Erera, Mike McCandless, Simon Willnauer)
 
@@ -10794,7 +10796,7 @@ API Changes
   can inspect the FieldInfo yourself to see if norms are present, which
   also allows you to get the type.  (Robert Muir)
 
-* LUCENE-2606: Changed RegexCapabilities interface to fix thread 
+* LUCENE-2606: Changed RegexCapabilities interface to fix thread
   safety, serialization, and performance problems. If you have
   written a custom RegexCapabilities it will need to be updated
   to the new API.  (Robert Muir, Uwe Schindler)
@@ -10804,7 +10806,7 @@ API Changes
 
 * LUCENE-2912: The field-specific hashmaps in SweetSpotSimilarity were removed.
   Instead, use PerFieldSimilarityWrapper to return different SweetSpotSimilaritys
-  for different fields, this way all parameters (such as TF factors) can be 
+  for different fields, this way all parameters (such as TF factors) can be
   customized on a per-field basis.  (Robert Muir)
 
 * LUCENE-3308: DuplicateFilter keepMode and processingMode have been converted to
@@ -10812,7 +10814,7 @@ API Changes
 
 * LUCENE-3483: Move Function grouping collectors from Solr to grouping module.
   (Martijn van Groningen)
-   
+
 * LUCENE-3606: FieldNormModifier was deprecated, because IndexReader's
   setNorm() was deprecated. Furthermore, this class is broken, as it does
   not take position overlaps into account while recalculating norms.
@@ -10827,14 +10829,14 @@ API Changes
 * LUCENE-3820: Deprecated constructors accepting pattern matching bounds. The input
   is buffered and matched in one pass. (Dawid Weiss)
 
-* LUCENE-2413: Deprecated PatternAnalyzer in common/miscellaneous, in favor 
+* LUCENE-2413: Deprecated PatternAnalyzer in common/miscellaneous, in favor
   of the pattern package (CharFilter, Tokenizer, TokenFilter).  (Robert Muir)
 
 * LUCENE-2413: Removed the AnalyzerUtil in common/miscellaneous.  (Robert Muir)
 
 * LUCENE-1370: Added ShingleFilter option to output unigrams if no shingles
   can be generated. (Chris Harris via Steven Rowe)
-   
+
 * LUCENE-2514, LUCENE-2551: JDK and ICU CollationKeyAnalyzers were changed to
   use pure byte keys when Version >= 4.0. This cuts sort key size approximately
   in half. (Robert Muir)
@@ -10863,8 +10865,8 @@ API Changes
 
 * LUCENE-4122: Replace Payload with BytesRef. (Andrzej Bialecki)
 
-* LUCENE-4132: IndexWriter.getConfig() now returns a LiveIndexWriterConfig object 
-  which can be used to change the IndexWriter's live settings. IndexWriterConfig 
+* LUCENE-4132: IndexWriter.getConfig() now returns a LiveIndexWriterConfig object
+  which can be used to change the IndexWriter's live settings. IndexWriterConfig
   is used only for initializing the IndexWriter. (Shai Erera)
 
 * LUCENE-3866: IndexReaderContext.leaves() is now the preferred way to access
@@ -10874,13 +10876,13 @@ API Changes
 New features
 
 * LUCENE-2604: Added RegexpQuery support to QueryParser. Regular expressions
-  are directly supported by the standard queryparser via 
+  are directly supported by the standard queryparser via
      fieldName:/expression/ OR /expression against default field/
-  Users who wish to search for literal "/" characters are advised to 
-  backslash-escape or quote those characters as needed. 
+  Users who wish to search for literal "/" characters are advised to
+  backslash-escape or quote those characters as needed.
   (Simon Willnauer, Robert Muir)
 
-* LUCENE-1606, LUCENE-2089: Adds AutomatonQuery, a MultiTermQuery that 
+* LUCENE-1606, LUCENE-2089: Adds AutomatonQuery, a MultiTermQuery that
   matches terms against a finite-state machine. Implement WildcardQuery
   and FuzzyQuery with finite-state methods. Adds RegexpQuery.
   (Robert Muir, Mike McCandless, Uwe Schindler, Mark Miller)
@@ -10910,7 +10912,7 @@ New features
   data and payloads in 5 separate files instead of the 2 used by
   standard codec), and int block (really a "base" for using
   block-based compressors like PForDelta for storing postings data).
-  
+
 * LUCENE-1458, LUCENE-2111: The in-memory terms index used by standard
   codec is more RAM efficient: terms data is stored as block byte
   arrays and packed integers.  Net RAM reduction for indexes that have
@@ -10935,10 +10937,10 @@ New features
   FieldComparator instance.  You can "return this", to just reuse the
   same instance, or you can return a comparator optimized to the new
   segment.  (yonik, Mike McCandless)
-  
+
 * LUCENE-2648: PackedInts.Iterator now supports to advance by more than a
-  single ordinal. (Simon Willnauer) 
-  
+  single ordinal. (Simon Willnauer)
+
 * LUCENE-2649: Objects in the FieldCache can optionally store Bits
   that mark which docs have real values in the native[] (ryan)
 
@@ -10965,7 +10967,7 @@ New features
   to use the new getRangeQuery method.  (Andrew Schurman, Mark Miller, yonik)
 
 * LUCENE-2742: Add native per-field postings format support. Codec lets you now
-  register a postings format for each field and which is in turn recorded 
+  register a postings format for each field and which is in turn recorded
   into the index. Postings formats are maintained on a per-segment basis and be
   resolved without knowing the actual postings format used for writing the segment.
   (Simon Willnauer)
@@ -10991,31 +10993,31 @@ New features
   that it inverts the index to compute the ords, but differs in that
   it's able to handle multi-valued fields and does not hold the term
   bytes in RAM. (Mike McCandless)
-  
-* LUCENE-3108, LUCENE-2935, LUCENE-2168, LUCENE-1231: Changes from 
+
+* LUCENE-3108, LUCENE-2935, LUCENE-2168, LUCENE-1231: Changes from
   DocValues (ColumnStrideFields):
-  
+
   - IndexWriter now supports typesafe dense per-document values stored in
     a column like storage. DocValues are stored on a per-document
     basis where each documents field can hold exactly one value of a given
     type. DocValues are provided via Fieldable and can be used in
     conjunction with stored and indexed values.
-     
+
   - DocValues provides an entirely RAM resident document id to value
     mapping per field as well as a DocIdSetIterator based disk-resident
     sequential access API relying on filesystem-caches.
-    
+
   - Both APIs are exposed via IndexReader and the Codec / Flex API allowing
     expert users to integrate customized DocValues reader and writer
     implementations by extending existing Codecs.
-    
+
   - DocValues provides implementations for primitive datatypes like int,
     long, float, double and arrays of byte. Byte based implementations further
     provide storage variants like straight or dereferenced stored bytes, fixed
-    and variable length bytes as well as index time sorted based on 
+    and variable length bytes as well as index time sorted based on
     user-provided comparators.
-  
-  (Mike McCandless, Simon Willnauer)  
+
+  (Mike McCandless, Simon Willnauer)
 
 * LUCENE-3209: Added MemoryCodec, which stores all terms & postings in
   RAM as an FST; this is good for primary-key fields if you frequently
@@ -11031,7 +11033,7 @@ New features
   on searching, NRT reopen time, etc.  (Mike McCandless)
 
 * LUCENE-2793: Directory#createOutput & Directory#openInput now accept an
-  IOContext instead of a buffer size to allow low level optimizations for 
+  IOContext instead of a buffer size to allow low level optimizations for
   different usecases like merging, flushing and reading.
   (Simon Willnauer, Mike McCandless, Varun Thacker)
 
@@ -11045,40 +11047,40 @@ New features
 
 * LUCENE-2959: Added a variety of different relevance ranking systems to Lucene.
 
-  - Added Okapi BM25, Language Models, Divergence from Randomness, and 
-    Information-Based Models. The models are pluggable, support all of lucene's 
+  - Added Okapi BM25, Language Models, Divergence from Randomness, and
+    Information-Based Models. The models are pluggable, support all of lucene's
     features (boosts, slops, explanations, etc) and queries (spans, etc).
 
-  - All models default to the same index-time norm encoding as 
-    DefaultSimilarity, so you can easily try these out/switch back and 
-    forth/run experiments and comparisons without reindexing. Note: most of 
-    the models do rely upon index statistics that are new in Lucene 4.0, so 
-    for existing 3.x indexes it's a good idea to upgrade your index to the 
+  - All models default to the same index-time norm encoding as
+    DefaultSimilarity, so you can easily try these out/switch back and
+    forth/run experiments and comparisons without reindexing. Note: most of
+    the models do rely upon index statistics that are new in Lucene 4.0, so
+    for existing 3.x indexes it's a good idea to upgrade your index to the
     new format with IndexUpgrader first.
 
-  - Added a new subclass SimilarityBase which provides a simplified API 
+  - Added a new subclass SimilarityBase which provides a simplified API
     for plugging in new ranking algorithms without dealing with all of the
-    nuances and implementation details of Lucene. 
+    nuances and implementation details of Lucene.
 
-  - For example, to use BM25 for all fields: 
+  - For example, to use BM25 for all fields:
      searcher.setSimilarity(new BM25Similarity());
 
-    If you instead want to apply different similarities (e.g. ones with 
+    If you instead want to apply different similarities (e.g. ones with
     different parameter values or different algorithms entirely) to different
     fields, implement PerFieldSimilarityWrapper with your per-field logic.
 
   (David Mark Nemeskey via Robert Muir)
 
-* LUCENE-3396: ReusableAnalyzerBase now provides a ReuseStrategy abstraction 
-  which controls how TokenStreamComponents are reused per request.  Two 
+* LUCENE-3396: ReusableAnalyzerBase now provides a ReuseStrategy abstraction
+  which controls how TokenStreamComponents are reused per request.  Two
   implementations are provided - GlobalReuseStrategy which implements the
-  current behavior of sharing components between all fields, and 
+  current behavior of sharing components between all fields, and
   PerFieldReuseStrategy which shares per field.  (Chris Male)
-  
-* LUCENE-2309: Added IndexableField.tokenStream(Analyzer) which is now 
-  responsible for creating the TokenStreams for Fields when they are to 
+
+* LUCENE-2309: Added IndexableField.tokenStream(Analyzer) which is now
+  responsible for creating the TokenStreams for Fields when they are to
   be indexed.  (Chris Male)
-  
+
 * LUCENE-3433: Added random access for non RAM resident IndexDocValues. RAM
   resident and disk resident IndexDocValues are now exposed via the Source
   interface. ValuesEnum has been removed in favour of Source. (Simon Willnauer)
@@ -11099,14 +11101,14 @@ New features
 * LUCENE-3638: Added sugar methods to IndexReader and IndexSearcher to
   load only certain fields when loading a document.  (Peter Chang via
   Mike McCandless)
-  
+
 * LUCENE-3628: Norms are represented as DocValues. AtomicReader exposes
-  a #normValues(String) method to obtain norms per field. (Simon Willnauer) 
+  a #normValues(String) method to obtain norms per field. (Simon Willnauer)
 
 * LUCENE-3687: Similarity#computeNorm(FieldInvertState, Norm) allows to compute
   norm values or arbitrary precision. Instead of returning a fixed single byte
   value, custom similarities can now set a integer, float or byte value to the
-  given Norm object. (Simon Willnauer)  
+  given Norm object. (Simon Willnauer)
 
 * LUCENE-2604, LUCENE-4103: Added RegexpQuery support to contrib/queryparser.
   (Simon Willnauer, Robert Muir, Daniel Truemper)
@@ -11121,8 +11123,8 @@ New features
 
 * LUCENE-2608: Added the ability to specify the accuracy at method time in the SpellChecker.  The per class
   method is also still available.  (Grant Ingersoll)
-    
-* LUCENE-2507: Added DirectSpellChecker, which retrieves correction candidates directly 
+
+* LUCENE-2507: Added DirectSpellChecker, which retrieves correction candidates directly
   from the term dictionary using levenshtein automata.  (Robert Muir)
 
 * LUCENE-3527: Add LuceneLevenshteinDistance, which computes string distance in a compatible
@@ -11151,29 +11153,29 @@ New features
 * LUCENE-3778: Added a grouping utility class that makes it easier to use result
   grouping for pure Lucene apps. (Martijn van Groningen)
 
-* LUCENE-2341: A new analysis/ filter: Morfologik - a dictionary-driven lemmatizer 
+* LUCENE-2341: A new analysis/ filter: Morfologik - a dictionary-driven lemmatizer
   (accurate stemmer) for Polish (includes morphosyntactic annotations).
   (Michał Dybizbański, Dawid Weiss)
 
-* LUCENE-2413: Consolidated Lucene/Solr analysis components into analysis/common. 
+* LUCENE-2413: Consolidated Lucene/Solr analysis components into analysis/common.
   New features from Solr now available to Lucene users include:
    - o.a.l.analysis.commongrams: Constructs n-grams for frequently occurring terms
-     and phrases. 
-   - o.a.l.analysis.charfilter.HTMLStripCharFilter: CharFilter that strips HTML 
+     and phrases.
+   - o.a.l.analysis.charfilter.HTMLStripCharFilter: CharFilter that strips HTML
      constructs.
-   - o.a.l.analysis.miscellaneous.WordDelimiterFilter: TokenFilter that splits words 
+   - o.a.l.analysis.miscellaneous.WordDelimiterFilter: TokenFilter that splits words
      into subwords and performs optional transformations on subword groups.
-   - o.a.l.analysis.miscellaneous.RemoveDuplicatesTokenFilter: TokenFilter which 
+   - o.a.l.analysis.miscellaneous.RemoveDuplicatesTokenFilter: TokenFilter which
      filters out Tokens at the same position and Term text as the previous token.
-   - o.a.l.analysis.miscellaneous.TrimFilter: Trims leading and trailing whitespace 
+   - o.a.l.analysis.miscellaneous.TrimFilter: Trims leading and trailing whitespace
      from Tokens in the stream.
-   - o.a.l.analysis.miscellaneous.KeepWordFilter: A TokenFilter that only keeps tokens 
+   - o.a.l.analysis.miscellaneous.KeepWordFilter: A TokenFilter that only keeps tokens
      with text contained in the required words (inverse of StopFilter).
-   - o.a.l.analysis.miscellaneous.HyphenatedWordsFilter: A TokenFilter that puts 
+   - o.a.l.analysis.miscellaneous.HyphenatedWordsFilter: A TokenFilter that puts
      hyphenated words broken into two lines back together.
    - o.a.l.analysis.miscellaneous.CapitalizationFilter: A TokenFilter that applies
      capitalization rules to tokens.
-   - o.a.l.analysis.pattern: Package for pattern-based analysis, containing a 
+   - o.a.l.analysis.pattern: Package for pattern-based analysis, containing a
      CharFilter, Tokenizer, and TokenFilter for transforming text with regexes.
    - o.a.l.analysis.synonym.SynonymFilter: A synonym filter that supports multi-word
      synonyms.
@@ -11217,10 +11219,10 @@ New features
 * LUCENE-4004: Add DisjunctionMaxQuery support to the xml query parser.
   (Benson Margulies via Robert Muir)
 
-* LUCENE-4025: Add maybeRefreshBlocking to ReferenceManager, to let a caller 
+* LUCENE-4025: Add maybeRefreshBlocking to ReferenceManager, to let a caller
   block until the refresh logic has been executed. (Shai Erera, Mike McCandless)
-  
-* LUCENE-4039: Add AddIndexesTask to benchmark, which uses IW.addIndexes. 
+
+* LUCENE-4039: Add AddIndexesTask to benchmark, which uses IW.addIndexes.
   (Shai Erera)
 
 * LUCENE-3514: Added IndexSearcher.searchAfter when Sort is used,
@@ -11229,15 +11231,15 @@ New features
 
 * LUCENE-4043: Added scoring support via score mode for query time joining.
   (Martijn van Groningen, Mike McCandless)
-  
-* LUCENE-3523: Added oal.search.spell.WordBreakSpellChecker, which 
-    generates suggestions by combining two or more terms and/or 
+
+* LUCENE-3523: Added oal.search.spell.WordBreakSpellChecker, which
+    generates suggestions by combining two or more terms and/or
     breaking terms into multiple words.  See Javadocs for usage. (James Dyer)
 
 * LUCENE-4019: Added improved parsing of Hunspell Dictionaries so those
-  rules missing the required number of parameters either ignored or 
+  rules missing the required number of parameters either ignored or
   cause a ParseException (depending on whether strict parsing is enabled).
-  (Luca Cavanna via Chris Male) 
+  (Luca Cavanna via Chris Male)
 
 * LUCENE-3440: Add ordered fragments feature with IDF-weighted terms for FVH.
   (Sebastian Lutze via Koji Sekiguchi)
@@ -11266,7 +11268,7 @@ Optimizations
   not seek backwards when a sub-range has no terms. It now only seeks
   when the current term is less than the next sub-range's lower end.
   (Uwe Schindler, Mike McCandless)
-  
+
 * LUCENE-2694: Optimize MultiTermQuery to be single pass for Term lookups.
   MultiTermQuery now stores TermState per leaf reader during rewrite to re-
   seek the term dictionary in TermQuery / TermWeight.
@@ -11278,9 +11280,9 @@ Optimizations
   Willnauer, Mike McCandless)
 
 * LUCENE-3328: BooleanQuery now uses a specialized ConjunctionScorer if all
-  boolean clauses are required and instances of TermQuery. 
+  boolean clauses are required and instances of TermQuery.
   (Simon Willnauer, Robert Muir)
-  
+
 * LUCENE-3643: FilteredQuery and IndexSearcher.search(Query, Filter,...)
   now optimize the special case query instanceof MatchAllDocsQuery to
   execute as ConstantScoreQuery.  (Uwe Schindler)
@@ -11288,10 +11290,10 @@ Optimizations
 * LUCENE-3509: Added fasterButMoreRam option for docvalues. This option controls whether the space for packed ints
   should be rounded up for better performance. This option only applies for docvalues types bytes fixed sorted
   and bytes var sorted. (Simon Willnauer, Martijn van Groningen)
-  
-* LUCENE-3795: Replace contrib/spatial with modules/spatial.  This includes 
+
+* LUCENE-3795: Replace contrib/spatial with modules/spatial.  This includes
   a basic spatial strategy interface.  (David Smiley, Chris Male, ryan)
-    
+
 * LUCENE-3932: Lucene3x codec loads terms index faster, by
   pre-allocating the packed ints array based on the .tii file size
   (Sean Bridges via Mike McCandless)
@@ -11318,7 +11320,7 @@ Optimizations
 
 * LUCENE-4098: Add bulk get/set methods to PackedInts (Adrien Grand
   via Mike McCandless)
-  
+
 * LUCENE-4156: DirectoryTaxonomyWriter.getSize is no longer synchronized.
   (Shai Erera, Sivan Yogev)
 
@@ -11361,25 +11363,25 @@ Bug fixes
 * LUCENE-3971: MappingCharFilter could return invalid final token position.
   (Dawid Weiss, Robert Muir)
 
-* LUCENE-3820: PatternReplaceCharFilter could return invalid token positions. 
+* LUCENE-3820: PatternReplaceCharFilter could return invalid token positions.
   (Dawid Weiss)
 
 * LUCENE-3969: Throw IAE on bad arguments that could cause confusing errors in
   CompoundWordTokenFilterBase, PatternTokenizer, PositionFilter,
-  SnowballFilter, PathHierarchyTokenizer, ReversePathHierarchyTokenizer, 
-  WikipediaTokenizer, and KeywordTokenizer. ShingleFilter and 
+  SnowballFilter, PathHierarchyTokenizer, ReversePathHierarchyTokenizer,
+  WikipediaTokenizer, and KeywordTokenizer. ShingleFilter and
   CommonGramsFilter now populate PositionLengthAttribute. Fixed
   PathHierarchyTokenizer to reset() all state. Protect against AIOOBE in
   ReversePathHierarchyTokenizer if skip is large. Fixed wrong final
-  offset calculation in PathHierarchyTokenizer. 
+  offset calculation in PathHierarchyTokenizer.
   (Mike McCandless, Uwe Schindler, Robert Muir)
 
-* LUCENE-4060: Fix a synchronization bug in 
+* LUCENE-4060: Fix a synchronization bug in
   DirectoryTaxonomyWriter.addTaxonomies(). Also, the method has been renamed to
   addTaxonomy and now takes only one Directory and one OrdinalMap.
   (Shai Erera, Gilad Barkai)
 
-* LUCENE-3590: Fix AIOOBE in BytesRef/CharsRef copyBytes/copyChars when 
+* LUCENE-3590: Fix AIOOBE in BytesRef/CharsRef copyBytes/copyChars when
   offset is nonzero, fix off-by-one in CharsRef.subSequence, and fix
   CharsRef's CharSequence methods to throw exceptions in boundary cases
   to properly meet the specification.  (Robert Muir)
@@ -11417,7 +11419,7 @@ Build
 * LUCENE-4047: Cleanup of LuceneTestCase: moved blocks of initialization/ cleanup
   code into JUnit instance and class rules. (Dawid Weiss)
 
-* LUCENE-4016: Require ANT 1.8.2+ for the build. 
+* LUCENE-4016: Require ANT 1.8.2+ for the build.
 
 * LUCENE-3808: Refactoring of testing infrastructure to use randomizedtesting
   package: http://labs.carrotsearch.com/randomizedtesting.html (Dawid Weiss)
@@ -11434,7 +11436,7 @@ Build
 
 * LUCENE-3285: Moved contrib/xml-query-parser's demo into lucene/demo
 
-* LUCENE-3271: Moved contrib/queries BooleanFilter, BoostingQuery, 
+* LUCENE-3271: Moved contrib/queries BooleanFilter, BoostingQuery,
   ChainedFilter, FilterClause and TermsFilter into lucene/queries
 
 * LUCENE-3381: Moved contrib/queries regex.*, DuplicateFilter,
@@ -11495,14 +11497,14 @@ Documentation
 
 
 ======================= Lucene 3.6.1 =======================
-More information about this release, including any errata related to the 
+More information about this release, including any errata related to the
 release notes, upgrade instructions, or other changes may be found online at:
    https://wiki.apache.org/lucene-java/Lucene3.6.1
 
 Bug Fixes
 
-* LUCENE-3969: Throw IAE on bad arguments that could cause confusing 
-  errors in KeywordTokenizer. 
+* LUCENE-3969: Throw IAE on bad arguments that could cause confusing
+  errors in KeywordTokenizer.
   (Uwe Schindler, Mike McCandless, Robert Muir)
 
 * LUCENE-3971: MappingCharFilter could return invalid final token position.
@@ -11513,7 +11515,7 @@ Bug Fixes
 
 * LUCENE-2566: + - operators allow any amount of whitespace (yonik, janhoy)
 
-* LUCENE-3590: Fix AIOOBE in BytesRef/CharsRef copyBytes/copyChars when 
+* LUCENE-3590: Fix AIOOBE in BytesRef/CharsRef copyBytes/copyChars when
   offset is nonzero, fix off-by-one in CharsRef.subSequence, and fix
   CharsRef's CharSequence methods to throw exceptions in boundary cases
   to properly meet the specification.  (Robert Muir)
@@ -11539,12 +11541,12 @@ Tests
 * LUCENE-3873: add MockGraphTokenFilter, testing analyzers with
   random graph tokens.  (Mike McCandless)
 
-* LUCENE-3968: factor out LookaheadTokenFilter from 
+* LUCENE-3968: factor out LookaheadTokenFilter from
   MockGraphTokenFilter (Mike McCandless)
 
 
 ======================= Lucene 3.6.0 =======================
-More information about this release, including any errata related to the 
+More information about this release, including any errata related to the
 release notes, upgrade instructions, or other changes may be found online at:
    https://wiki.apache.org/lucene-java/Lucene3.6
 
@@ -11554,11 +11556,11 @@ Changes in backwards compatibility policy
   FieldCacheTermsFilter.FieldCacheTermsFilterDocIdSet was removed and
   replaced by another internal implementation.  (Uwe Schindler)
 
-* LUCENE-3620: FilterIndexReader now overrides all methods of IndexReader that 
-  it should (note that some are still not overridden, as they should be 
-  overridden by sub-classes only). In the process, some methods of IndexReader 
+* LUCENE-3620: FilterIndexReader now overrides all methods of IndexReader that
+  it should (note that some are still not overridden, as they should be
+  overridden by sub-classes only). In the process, some methods of IndexReader
   were made final. This is not expected to affect many apps, since these methods
-  already delegate to abstract methods, which you had to already override 
+  already delegate to abstract methods, which you had to already override
   anyway. (Shai Erera)
 
 * LUCENE-3636: Added SearcherFactory, used by SearcherManager and NRTManager
@@ -11573,7 +11575,7 @@ Changes in backwards compatibility policy
   the owner of the reader core, and other ambiguities). The API is revised:
   You can set ReaderClosedListeners on any IndexReader, and onClose is called
   when that reader is closed.  SegmentReader has CoreClosedListeners that you
-  can register to know when a shared reader core is closed.  
+  can register to know when a shared reader core is closed.
   (Uwe Schindler, Mike McCandless, Robert Muir)
 
 * LUCENE-3652: The package org.apache.lucene.messages was moved to
@@ -11597,12 +11599,12 @@ Changes in backwards compatibility policy
 * LUCENE-3701: Added an additional argument to the expert FST.Builder
   ctor to take FreezeTail, which you can use to (very-expertly) customize
   the FST construction process. Pass null if you want the default
-  behavior.  Added seekExact() to FSTEnum, and added FST.save/read 
+  behavior.  Added seekExact() to FSTEnum, and added FST.save/read
   from a File. (Mike McCandless, Dawid Weiss, Robert Muir)
-  
+
 * LUCENE-3712: Removed unused and untested ReaderUtil#subReader methods.
   (Uwe Schindler)
-  
+
 * LUCENE-3672: Deprecate Directory.fileModified,
   IndexCommit.getTimestamp and .getVersion and
   IndexReader.lastModified and getCurrentVersion (Andrzej Bialecki,
@@ -11612,7 +11614,7 @@ Changes in backwards compatibility policy
   methods getCurrentVersion and getCommitUserData, and non-static
   method getCommitUserData (use getIndexCommit().getUserData()
   instead).  (Ryan McKinley, Robert Muir, Mike McCandless)
-  
+
 * LUCENE-3867: Deprecate instance creation of RamUsageEstimator, instead
   the new static method sizeOf(Object) should be used. As the algorithm
   is now using Hotspot(TM) internals (reference size, header sizes,
@@ -11621,7 +11623,7 @@ Changes in backwards compatibility policy
   no longer support String intern-ness checking, interned strings
   now count to memory usage as any other Java object.
   (Dawid Weiss, Uwe Schindler, Shai Erera)
-  
+
 * LUCENE-3738: All readXxx methods in BufferedIndexInput were made
   final. Subclasses should only override protected readInternal /
   seekInternal.  (Uwe Schindler)
@@ -11629,26 +11631,26 @@ Changes in backwards compatibility policy
 * LUCENE-2599: Deprecated the spatial contrib module, which was buggy and not
   well maintained.  Lucene 4 includes a new spatial module that replaces this.
   (David Smiley, Ryan McKinley, Chris Male)
-  
+
 Changes in Runtime Behavior
 
 * LUCENE-3796, SOLR-3241: Throw an exception if you try to set an index-time
   boost on a field that omits norms. Because the index-time boost
-  is multiplied into the norm, previously your boost would be 
+  is multiplied into the norm, previously your boost would be
   silently discarded.  (Tomás Fernández Löbbe, Hoss Man, Robert Muir)
 
 * LUCENE-3848: Fix tokenstreams to not produce a stream with an initial
   position increment of 0: which is out of bounds (overlapping with a
   non-existent previous term). Consumers such as IndexWriter and QueryParser
-  still check for and silently correct this situation today, but at some point 
+  still check for and silently correct this situation today, but at some point
   in the future they may throw an exception.  (Mike McCandless, Robert Muir)
-  
+
 * LUCENE-3738: DataInput/DataOutput no longer allow negative vLongs. Negative
   vInts are still supported (for index backwards compatibility), but
   should not be used in new code. The read method for negative vLongs
   was already broken since Lucene 3.1.
   (Uwe Schindler, Mike McCandless, Robert Muir)
-  
+
 Security fixes
 
 * LUCENE-3588: Try harder to prevent SIGSEGV on cloned MMapIndexInputs:
@@ -11686,7 +11688,7 @@ API Changes
 
 * LUCENE-3766: Deprecate no-arg ctors of Tokenizer. Tokenizers are
   TokenStreams with Readers: tokenizers with null Readers will not be
-  supported in Lucene 4.0, just use a TokenStream.  
+  supported in Lucene 4.0, just use a TokenStream.
   (Mike McCandless, Robert Muir)
 
 * LUCENE-3769: Simplified NRTManager by requiring applyDeletes to be
@@ -11695,13 +11697,13 @@ API Changes
   never applying deletes).  (MJB, Shai Erera, Mike McCandless)
 
 * LUCENE-3761: Generalize SearcherManager into an abstract ReferenceManager.
-  SearcherManager remains a concrete class, but due to the refactoring, the 
+  SearcherManager remains a concrete class, but due to the refactoring, the
   method maybeReopen has been deprecated in favor of maybeRefresh().
   (Shai Erera, Mike McCandless, Simon Willnauer)
 
 * LUCENE-3776: You now acquire/release the IndexSearcher directly from
   NRTManager.  (Mike McCandless)
-  
+
 New Features
 
 * LUCENE-3593: Added a FieldValueFilter that accepts all documents that either
@@ -11720,12 +11722,12 @@ New Features
   queries.  Literal asterisks may be represented by quoting or escaping
   (i.e. \* or "*")  Custom QueryParser subclasses overriding getRangeQuery()
   will be passed null for any open endpoint. (Ingo Renner, Adriano
-  Crestani, yonik, Mike McCandless 
+  Crestani, yonik, Mike McCandless
 
 * LUCENE-3121: Add sugar reverse lookup (given an output, find the
   input mapping to it) for FSTs that have strictly monotonic long
   outputs (such as an ord).  (Mike McCandless)
-  
+
 * LUCENE-3671: Add TypeTokenFilter that filters tokens based on
   their TypeAttribute.  (Tommaso Teofili via Uwe Schindler)
 
@@ -11740,7 +11742,7 @@ New Features
 
 * LUCENE-3789: Expose MTQ TermsEnum via RewriteMethod for non package private
   access (Simon Willnauer)
-  
+
 * LUCENE-3881: Added UAX29URLEmailAnalyzer: a standard analyzer that recognizes
   URLs and emails. (Steve Rowe)
 
@@ -11762,10 +11764,10 @@ Bug fixes
 * LUCENE-3641: Fixed MultiReader to correctly propagate readerFinishedListeners
   to clones/reopened readers.  (Uwe Schindler)
 
-* LUCENE-3642, SOLR-2891, LUCENE-3717: Fixed bugs in CharTokenizer, n-gram tokenizers/filters, 
-  compound token filters, thai word filter, icutokenizer, pattern analyzer, 
-  wikipediatokenizer, and smart chinese where they would create invalid offsets in 
-  some situations, leading to problems in highlighting.  
+* LUCENE-3642, SOLR-2891, LUCENE-3717: Fixed bugs in CharTokenizer, n-gram tokenizers/filters,
+  compound token filters, thai word filter, icutokenizer, pattern analyzer,
+  wikipediatokenizer, and smart chinese where they would create invalid offsets in
+  some situations, leading to problems in highlighting.
   (Max Beutel, Edwin Steiner via Robert Muir)
 
 * LUCENE-3639: TopDocs.merge was incorrectly setting TopDocs.maxScore to
@@ -11797,10 +11799,10 @@ Bug fixes
   from the delegate DocIdSet.iterator(), which is allowed to return
   null by DocIdSet specification when no documents match.
   (Shay Banon via Uwe Schindler)
-  
+
 * LUCENE-3821: SloppyPhraseScorer missed documents that ExactPhraseScorer finds
-  When phrase query had repeating terms (e.g. "yes no yes")  
-  sloppy query missed documents that exact query matched. 
+  When phrase query had repeating terms (e.g. "yes no yes")
+  sloppy query missed documents that exact query matched.
   Fixed except when for repeating multiterms (e.g. "yes no yes|no").
   (Robert Muir, Doron Cohen)
 
@@ -11819,12 +11821,12 @@ Bug fixes
   with a StopFilter. (Robert Muir)
 
 * LUCENE-3876: Fix bug where positions for a document exceeding
-  Integer.MAX_VALUE/2 would produce a corrupt index.  
+  Integer.MAX_VALUE/2 would produce a corrupt index.
   (Simon Willnauer, Mike McCandless, Robert Muir)
 
 * LUCENE-3880: UAX29URLEmailTokenizer now recognizes emails when the mailto:
   scheme is prepended. (Kai Gülzau, Steve Rowe)
-    
+
 Optimizations
 
 * LUCENE-3653: Improve concurrency in VirtualMethod and AttributeSource by
@@ -11839,7 +11841,7 @@ Documentation
 * LUCENE-3926: Improve documentation of RAMDirectory, because this
   class is not intended to work with huge indexes. Everything beyond
   several hundred megabytes will waste resources (GC cycles), because
-  it uses an internal buffer size of 1024 bytes, producing millions of 
+  it uses an internal buffer size of 1024 bytes, producing millions of
   byte[1024] arrays. This class is optimized for small memory-resident
   indexes. It also has bad concurrency on multithreaded environments.
   It is recommended to materialize large indexes on disk and use
@@ -11850,10 +11852,10 @@ Documentation
 
 Build
 
-* LUCENE-3857: exceptions from other threads in beforeclass/etc do not fail 
+* LUCENE-3857: exceptions from other threads in beforeclass/etc do not fail
   the test (Dawid Weiss)
 
-* LUCENE-3847: LuceneTestCase will now check for modifications of System 
+* LUCENE-3847: LuceneTestCase will now check for modifications of System
   properties before and after each test (and suite). If changes are detected,
   the test will fail. A rule can be used to reset system properties to
   before-scope state (and this has been used to make Solr tests pass).
@@ -11899,7 +11901,7 @@ Build
   by refactoring the build task into an ANT task and modifying build scripts
   to perform top-level checks. (Dawid Weiss, Steve Rowe, Robert Muir)
 
-* LUCENE-3762: Upgrade JUnit to 4.10, refactor state-machine of detecting 
+* LUCENE-3762: Upgrade JUnit to 4.10, refactor state-machine of detecting
   setUp/tearDown call chaining in LuceneTestCase. (Dawid Weiss, Robert Muir)
 
 * LUCENE-3944: Make the 'generate-maven-artifacts' target use filtered POMs
@@ -11929,14 +11931,14 @@ Changes in backwards compatibility policy
   (Uwe Schindler, Doron Cohen, Mike McCandless)
 
 * LUCENE-3541: Remove IndexInput's protected copyBuf. If you want to
-  keep a buffer in your IndexInput, do this yourself in your implementation, 
+  keep a buffer in your IndexInput, do this yourself in your implementation,
   and be sure to do the right thing on clone()!  (Robert Muir)
-  
+
 * LUCENE-2822: TimeLimitingCollector now expects a counter clock instead of
   relying on a private daemon thread. The global time limiting clock thread
   has been exposed and is now lazily loaded and fully optional.
   TimeLimitingCollector now supports setting clock baseline manually to include
-  prelude of a search. Previous versions set the baseline on construction time, 
+  prelude of a search. Previous versions set the baseline on construction time,
   now baseline is set once the first IndexReader is passed to the collector
   unless set before. (Simon Willnauer)
 
@@ -11949,7 +11951,7 @@ Changes in runtime behavior
 
 Bug fixes
 
-* LUCENE-3412: SloppyPhraseScorer was returning non-deterministic results 
+* LUCENE-3412: SloppyPhraseScorer was returning non-deterministic results
   for queries with many repeats (Doron Cohen)
 
 * LUCENE-3421: PayloadTermQuery's explain was wrong when includeSpanScore=false.
@@ -11958,7 +11960,7 @@ Bug fixes
 * LUCENE-3432: IndexWriter.expungeDeletes with TieredMergePolicy
   should ignore the maxMergedSegmentMB setting (v.sevel via Mike
   McCandless)
-  
+
 * LUCENE-3442: TermQuery.TermWeight.scorer() returns null for non-atomic
   IndexReaders (optimization bug, introcuced by LUCENE-2829), preventing
   QueryWrapperFilter and similar classes to get a top-level DocIdSet.
@@ -11974,7 +11976,7 @@ Bug fixes
   files) on the closed IndexWriter (Robert Muir, Mike McCandless)
 
 * LUCENE-3215: SloppyPhraseScorer sometimes computed Infinite freq
-  (Robert Muir, Doron Cohen)  
+  (Robert Muir, Doron Cohen)
 
 * LUCENE-3503: DisjunctionSumScorer would give slightly different scores
   for a document depending if you used nextDoc() versus advance().
@@ -11982,7 +11984,7 @@ Bug fixes
 
 * LUCENE-3529: Properly support indexing an empty field with empty term text.
   Previously, if you had assertions enabled you would receive an error during
-  flush, if you didn't, you would get an invalid index.  
+  flush, if you didn't, you would get an invalid index.
   (Mike McCandless, Robert Muir)
 
 * LUCENE-2633: PackedInts Packed32 and Packed64 did not support internal
@@ -11992,12 +11994,12 @@ Bug fixes
   error message in IndexFormatTooOldException was incorrect. (Uwe Schindler,
   Mike McCandless)
 
-* LUCENE-3541: IndexInput's default copyBytes() implementation was not safe 
+* LUCENE-3541: IndexInput's default copyBytes() implementation was not safe
   across multiple threads, because all clones shared the same buffer.
   (Robert Muir)
-  
+
 * LUCENE-3548: Fix CharsRef#append to extend length of the existing char[]
-  and preserve existing chars. (Simon Willnauer) 
+  and preserve existing chars. (Simon Willnauer)
 
 * LUCENE-3582: Normalize NaN values in NumericUtils.floatToSortableInt() /
   NumericUtils.doubleToSortableLong(), so this is consistent with stored
@@ -12042,24 +12044,24 @@ New Features
 * LUCENE-3448: Added FixedBitSet.and(other/DISI), andNot(other/DISI).
   (Uwe Schindler)
 
-* LUCENE-2215: Added IndexSearcher.searchAfter which returns results after a 
+* LUCENE-2215: Added IndexSearcher.searchAfter which returns results after a
   specified ScoreDoc (e.g. last document on the previous page) to support deep
-  paging use cases.  (Aaron McCurry, Grant Ingersoll, Robert Muir) 
+  paging use cases.  (Aaron McCurry, Grant Ingersoll, Robert Muir)
 
 * LUCENE-1990: Adds internal packed ints implementation, to be used
   for more efficient storage of int arrays when the values are
   bounded, for example for storing the terms dict index (Toke
   Eskildsen via Mike McCandless)
-  
+
 * LUCENE-3558: Moved SearcherManager, NRTManager & SearcherLifetimeManager into
   core. All classes are contained in o.a.l.search. (Simon Willnauer)
 
 Optimizations
 
-* LUCENE-3426: Add NGramPhraseQuery which extends PhraseQuery and tries to 
-  reduce the number of terms of the query when rewrite(), in order to improve 
+* LUCENE-3426: Add NGramPhraseQuery which extends PhraseQuery and tries to
+  reduce the number of terms of the query when rewrite(), in order to improve
   performance.  (Robert Muir, Koji Sekiguchi)
-  
+
 * LUCENE-3494: Optimize FilteredQuery to remove a multiply in score()
   (Uwe Schindler, Robert Muir)
 
@@ -12083,8 +12085,8 @@ Test Cases
   enabled. (Uwe Schindler)
 
 * LUCENE-3506: tests relying on assertions being enabled were no-op because
-  they ignored AssertionError. With this fix now entire test framework 
-  (every test) fails if assertions are disabled, unless 
+  they ignored AssertionError. With this fix now entire test framework
+  (every test) fails if assertions are disabled, unless
   -Dtests.asserts.gracious=true is specified. (Doron Cohen)
 
 Build
@@ -12120,15 +12122,15 @@ Bug fixes
 * LUCENE-3340: Fixed case where IndexWriter was not flushing at
   exactly maxBufferedDeleteTerms (Mike McCandless)
 
-* LUCENE-3358, LUCENE-3361: StandardTokenizer and UAX29URLEmailTokenizer 
-  wrongly discarded combining marks attached to Han or Hiragana characters, 
-  this is fixed if you supply Version >= 3.4 If you supply a previous 
-  lucene version, you get the old buggy behavior for backwards compatibility.  
+* LUCENE-3358, LUCENE-3361: StandardTokenizer and UAX29URLEmailTokenizer
+  wrongly discarded combining marks attached to Han or Hiragana characters,
+  this is fixed if you supply Version >= 3.4 If you supply a previous
+  lucene version, you get the old buggy behavior for backwards compatibility.
   (Trejkaz, Robert Muir)
 
 * LUCENE-3368: IndexWriter commits segments without applying their buffered
   deletes when flushing concurrently. (Simon Willnauer, Mike McCandless)
-  
+
 * LUCENE-3365: Create or Append mode determined before obtaining write lock
   can cause IndexWriter overriding an existing index.
   (Geoff Cooney via Simon Willnauer)
@@ -12146,10 +12148,10 @@ Bug fixes
   meaning an operating system or hardware crash, or power loss, could
   easily corrupt the index.  (Mark Miller, Robert Muir, Mike
   McCandless)
-  
+
 New Features
 
-* LUCENE-3290: Added FieldInvertState.numUniqueTerms 
+* LUCENE-3290: Added FieldInvertState.numUniqueTerms
   (Mike McCandless, Robert Muir)
 
 * LUCENE-3280: Add FixedBitSet, like OpenBitSet but is not elastic
@@ -12175,7 +12177,7 @@ New Features
 
 Optimizations
 
-* LUCENE-3201, LUCENE-3218: CompoundFileSystem code has been consolidated 
+* LUCENE-3201, LUCENE-3218: CompoundFileSystem code has been consolidated
   into a Directory implementation. Reading is optimized for MMapDirectory,
   NIOFSDirectory and SimpleFSDirectory to only map requested parts of the
   CFS into an IndexInput. Writing to a CFS now tries to append to the CF
@@ -12217,10 +12219,10 @@ Changes in backwards compatibility policy
   implementation. (Uwe Schindler, Robert Muir, Yonik Seeley)
 
 * LUCENE-3238: Made MultiTermQuery.rewrite() final, to prevent
-  problems (such as not properly setting rewrite methods, or 
+  problems (such as not properly setting rewrite methods, or
   not working correctly with things like SpanMultiTermQueryWrapper).
   To rewrite to a simpler form, instead return a simpler enum
-  from getEnum(IndexReader). For example, to rewrite to a single term, 
+  from getEnum(IndexReader). For example, to rewrite to a single term,
   return a SingleTermEnum.  (ludovic Boutros, Uwe Schindler, Robert Muir)
 
 Changes in runtime behavior
@@ -12229,7 +12231,7 @@ Changes in runtime behavior
   lock file is not stored in the index has changed.  This means you
   will see a different lucene-XXX-write.lock in your lock directory.
   (Robert Muir, Uwe Schindler, Mike McCandless)
-  
+
 * LUCENE-3146: IndexReader.setNorm throws IllegalStateException if the field
   does not store norms. (Shai Erera, Mike McCandless)
 
@@ -12237,7 +12239,7 @@ Changes in runtime behavior
   FSDirectory.open now defaults to MMapDirectory instead of
   NIOFSDirectory since MMapDirectory gives better performance.  (Mike
   McCandless)
-  
+
 * LUCENE-3200: MMapDirectory now uses chunk sizes that are powers of 2.
   When setting the chunk size, it is rounded down to the next possible
   value. The new default value for 64 bit platforms is 2^30 (1 GiB),
@@ -12245,11 +12247,11 @@ Changes in runtime behavior
   Internally, MMapDirectory now only uses one dedicated final IndexInput
   implementation supporting multiple chunks, which makes Hotspot's life
   easier.  (Uwe Schindler, Robert Muir, Mike McCandless)
-  
+
 Bug fixes
 
-* LUCENE-3147,LUCENE-3152: Fixed open file handles leaks in many places in the 
-  code. Now MockDirectoryWrapper (in test-framework) tracks all open files, 
+* LUCENE-3147,LUCENE-3152: Fixed open file handles leaks in many places in the
+  code. Now MockDirectoryWrapper (in test-framework) tracks all open files,
   including locks, and fails if the test fails to release all of them.
   (Mike McCandless, Robert Muir, Shai Erera, Simon Willnauer)
 
@@ -12295,7 +12297,7 @@ API Changes
 
 Optimizations
 
-* LUCENE-1736: DateTools.java general improvements. 
+* LUCENE-1736: DateTools.java general improvements.
   (David Smiley via Steve Rowe)
 
 New Features
@@ -12303,7 +12305,7 @@ New Features
 * LUCENE-3140: Added experimental FST implementation to Lucene.
   (Robert Muir, Dawid Weiss, Mike McCandless)
 
-* LUCENE-3193: A new TwoPhaseCommitTool allows running a 2-phase commit 
+* LUCENE-3193: A new TwoPhaseCommitTool allows running a 2-phase commit
   algorithm over objects that implement the new TwoPhaseCommit interface (such
   as IndexWriter). (Shai Erera)
 
@@ -12320,16 +12322,16 @@ Build
 
 * LUCENE-1344: Create OSGi bundle using dev-tools/maven.
   (Nicolas Lalevée, Luca Stancapiano via ryan)
-  
+
 * LUCENE-3204: The maven-ant-tasks jar is now included in the source tree;
   users of the generate-maven-artifacts target no longer have to manually
-  place this jar in the Ant classpath.  NOTE: when Ant looks for the 
+  place this jar in the Ant classpath.  NOTE: when Ant looks for the
   maven-ant-tasks jar, it looks first in its pre-existing classpath, so
-  any copies it finds will be used instead of the copy included in the 
+  any copies it finds will be used instead of the copy included in the
   Lucene/Solr source tree.  For this reason, it is recommeded to remove
   any copies of the maven-ant-tasks jar in the Ant classpath, e.g. under
-  ~/.ant/lib/ or under the Ant installation's lib/ directory. (Steve Rowe) 
-  
+  ~/.ant/lib/ or under the Ant installation's lib/ directory. (Steve Rowe)
+
 
 ======================= Lucene 3.2.0 =======================
 
@@ -12372,22 +12374,22 @@ Changes in runtime behavior
   segments. This means docIDs no longer necessarily stay "in order"
   during indexing.  If this is a problem then you can use either of
   the LogMergePolicy impls.  (Mike McCandless)
-  
+
 New features
 
 * LUCENE-3082: Added index upgrade tool oal.index.IndexUpgrader
   that allows to upgrade all segments to last recent supported index
   format without fully optimizing.  (Uwe Schindler, Mike McCandless)
 
-* LUCENE-1076: Added TieredMergePolicy which is able to merge non-contiguous 
+* LUCENE-1076: Added TieredMergePolicy which is able to merge non-contiguous
   segments, which means docIDs no longer necessarily stay "in order".
   (Mike McCandless, Shai Erera)
 
-* LUCENE-3071: Adding ReversePathHierarchyTokenizer, added skip parameter to 
+* LUCENE-3071: Adding ReversePathHierarchyTokenizer, added skip parameter to
   PathHierarchyTokenizer (Olivier Favre via ryan)
 
-* LUCENE-1421, LUCENE-3102: added CachingCollector which allow you to cache 
-  document IDs and scores encountered during the search, and "replay" them to 
+* LUCENE-1421, LUCENE-3102: added CachingCollector which allow you to cache
+  document IDs and scores encountered during the search, and "replay" them to
   another Collector. (Mike McCandless, Shai Erera)
 
 * LUCENE-3112: Added experimental IndexWriter.add/updateDocuments,
@@ -12397,7 +12399,7 @@ New features
 API Changes
 
 * LUCENE-3061: IndexWriter's getNextMerge() and merge(OneMerge) are now public
-  (though @lucene.experimental), allowing for custom MergeScheduler 
+  (though @lucene.experimental), allowing for custom MergeScheduler
   implementations. (Shai Erera)
 
 * LUCENE-3065: Document.getField() was deprecated, as it throws
@@ -12420,8 +12422,8 @@ Optimizations
 
 Bug fixes
 
-* LUCENE-2996: addIndexes(IndexReader) did not flush before adding the new 
-  indexes, causing existing deletions to be applied on the incoming indexes as 
+* LUCENE-2996: addIndexes(IndexReader) did not flush before adding the new
+  indexes, causing existing deletions to be applied on the incoming indexes as
   well. (Shai Erera, Mike McCandless)
 
 * LUCENE-3024: Index with more than 2.1B terms was hitting AIOOBE when
@@ -12431,7 +12433,7 @@ Bug fixes
 * LUCENE-3042: When a filter or consumer added Attributes to a TokenStream
   chain after it was already (partly) consumed [or clearAttributes(),
   captureState(), cloneAttributes(),... was called by the Tokenizer],
-  the Tokenizer calling clearAttributes() or capturing state after addition 
+  the Tokenizer calling clearAttributes() or capturing state after addition
   may not do this on the newly added Attribute. This bug affected only
   very special use cases of the TokenStream-API, most users would not
   have recognized it.  (Uwe Schindler, Robert Muir)
@@ -12441,7 +12443,7 @@ Bug fixes
   PhraseQuery as term with lower doc freq will also have less positions.
   (Uwe Schindler, Robert Muir, Otis Gospodnetic)
 
-* LUCENE-3068: sloppy phrase query failed to match valid documents when multiple 
+* LUCENE-3068: sloppy phrase query failed to match valid documents when multiple
   query terms had same position in the query. (Doron Cohen)
 
 * LUCENE-3012: Lucene writes the header now for separate norm files (*.sNNN)
@@ -12449,7 +12451,7 @@ Bug fixes
 
 Build
 
-* LUCENE-3006: Building javadocs will fail on warnings by default. 
+* LUCENE-3006: Building javadocs will fail on warnings by default.
   Override with -Dfailonjavadocwarning=false (sarowe, gsingers)
 
 * LUCENE-3128: "ant eclipse" creates a .project file for easier Eclipse
@@ -12457,8 +12459,8 @@ Build
 
 Test Cases
 
-* LUCENE-3002: added 'tests.iter.min' to control 'tests.iter' by allowing to 
-  stop iterating if at least 'tests.iter.min' ran and a failure occured. 
+* LUCENE-3002: added 'tests.iter.min' to control 'tests.iter' by allowing to
+  stop iterating if at least 'tests.iter.min' ran and a failure occured.
   (Shai Erera, Chris Hostetter)
 
 ======================= Lucene 3.1.0 =======================
@@ -13003,7 +13005,7 @@ New features
   (Tim Smith, Grant Ingersoll)
 
 * LUCENE-2692: Added several new SpanQuery classes for positional checking
-  (match is in a range, payload is a specific value) (Grant Ingersoll)  
+  (match is in a range, payload is a specific value) (Grant Ingersoll)
 
 Optimizations
 
@@ -13193,7 +13195,7 @@ Documentation
 
 * LUCENE-2894: Use google-code-prettify for syntax highlighting in javadoc.
   (Shinichiro Abe, Koji Sekiguchi)
-  
+
 ================== Release 2.9.4 / 3.0.3 ====================
 
 Changes in runtime behavior
@@ -13215,7 +13217,7 @@ Changes in runtime behavior
   compound format is on.  This change was made to reduce peak
   transient disk usage during optimize which increased due to
   LUCENE-2762.  (Mike McCandless)
-  
+
 Bug fixes
 
 * LUCENE-2142 (correct fix): FieldCacheImpl.getStringIndex no longer
@@ -13245,7 +13247,7 @@ Bug fixes
 
 * LUCENE-2593: Fixed certain rare cases where a disk full could lead
   to a corrupted index (Robert Muir, Mike McCandless)
-  
+
 * LUCENE-2620: Fixed a bug in WildcardQuery where too many asterisks
   would result in unbearably slow performance.  (Nick Barkas via Robert Muir)
 
@@ -13257,10 +13259,10 @@ Bug fixes
 
 * LUCENE-2650: Added extra safety to MMapIndexInput clones to prevent accessing
   an unmapped buffer if the input is closed (Mike McCandless, Uwe Schindler, Robert Muir)
-  
+
 * LUCENE-2384: Reset zzBuffer in StandardTokenizerImpl when lexer is reset.
-  (Ruben Laguna via Uwe Schindler, sub-issue of LUCENE-2074) 
-  
+  (Ruben Laguna via Uwe Schindler, sub-issue of LUCENE-2074)
+
 * LUCENE-2658: Exceptions while processing term vectors enabled for multiple
   fields could lead to invalid ArrayIndexOutOfBoundsExceptions.
   (Robert Muir, Mike McCandless)
@@ -13273,7 +13275,7 @@ Bug fixes
 
 * LUCENE-2549: Fix TimeLimitingCollector#TimeExceededException to record
   the absolute docid.  (Uwe Schindler)
- 
+
 * LUCENE-2533: fix FileSwitchDirectory.listAll to not return dups when
   primary & secondary dirs share the same underlying directory.
   (Michael McCandless)
@@ -13340,7 +13342,7 @@ New features
   have the same bugfix level, features, and the same index format with this slight
   compression difference. In general, Lucene does not support reading newer
   indexes with older library versions. (Uwe Schindler)
-  
+
 Documentation
 
 * LUCENE-2239: Documented limitations in NIOFSDirectory and MMapDirectory due to
@@ -13357,8 +13359,8 @@ Changes in backwards compatibility policy
 
 Changes in runtime behavior
 
-* LUCENE-2421: NativeFSLockFactory does not throw LockReleaseFailedException if 
-  it cannot delete the lock file, since obtaining the lock does not fail if the 
+* LUCENE-2421: NativeFSLockFactory does not throw LockReleaseFailedException if
+  it cannot delete the lock file, since obtaining the lock does not fail if the
   file is there. (Shai Erera)
 
 * LUCENE-2060 (2.9.3 only): Changed ConcurrentMergeScheduler's default for
@@ -13381,9 +13383,9 @@ Bug fixes
 * LUCENE-2142: FieldCacheImpl.getStringIndex no longer throws an
   exception when term count exceeds doc count.  (Mike McCandless)
 
-* LUCENE-2104: NativeFSLock.release() would silently fail if the lock is held by 
+* LUCENE-2104: NativeFSLock.release() would silently fail if the lock is held by
   another thread/process.  (Shai Erera via Uwe Schindler)
-  
+
 * LUCENE-2283: Use shared memory pool for term vector and stored
   fields buffers. This memory will be reclaimed if needed according to
   the configured RAM Buffer Size for the IndexWriter.  This also fixes
@@ -13403,9 +13405,9 @@ Bug fixes
   in IndexWriter, nor the Reader in Tokenizer after close is
   called.  (Ruben Laguna, Uwe Schindler, Mike McCandless)
 
-* LUCENE-2417: IndexCommit did not implement hashCode() and equals() 
+* LUCENE-2417: IndexCommit did not implement hashCode() and equals()
   consistently. Now they both take Directory and version into consideration. In
-  addition, all of IndexComnmit methods which threw 
+  addition, all of IndexComnmit methods which threw
   UnsupportedOperationException are now abstract. (Shai Erera)
 
 * LUCENE-2467: Fixed memory leaks in IndexWriter when large documents
@@ -13473,7 +13475,7 @@ API Changes
 Optimizations
 
 * LUCENE-2494 (3.0.2 only): Use CompletionService in ParallelMultiSearcher
-  instead of simple polling for results. (Edward Drapkin, Simon Willnauer) 
+  instead of simple polling for results. (Edward Drapkin, Simon Willnauer)
 
 * LUCENE-2135: On IndexReader.close, forcefully evict any entries from
   the FieldCache rather than waiting for the WeakHashMap to release
@@ -13500,7 +13502,7 @@ Changes in backwards compatibility policy
   from FuzzyQuery. The change was needed because the comparator of this
   class had to be changed in an incompatible way. The class was never
   intended to be public.  (Uwe Schindler, Mike McCandless)
-  
+
 Bug fixes
 
  * LUCENE-2092: BooleanQuery was ignoring disableCoord in its hashCode
@@ -13520,7 +13522,7 @@ Bug fixes
 
  * LUCENE-2158: At high indexing rates, NRT reader could temporarily
    lose deletions.  (Mike McCandless)
-  
+
  * LUCENE-2182: DEFAULT_ATTRIBUTE_FACTORY was failing to load
    implementation class when interface was loaded by a different
    class loader.  (Uwe Schindler, reported on java-user by Ahmed El-dawy)
@@ -13528,12 +13530,12 @@ Bug fixes
  * LUCENE-2257: Increase max number of unique terms in one segment to
    termIndexInterval (default 128) * ~2.1 billion = ~274 billion.
    (Tom Burton-West via Mike McCandless)
-  
+
  * LUCENE-2260: Fixed AttributeSource to not hold a strong
    reference to the Attribute/AttributeImpl classes which prevents
    unloading of custom attributes loaded by other classloaders
    (e.g. in Solr plugins).  (Uwe Schindler)
- 
+
  * LUCENE-1941: Fix Min/MaxPayloadFunction returns 0 when
    only one payload is present.  (Erik Hatcher, Mike McCandless
    via Uwe Schindler)
@@ -13586,7 +13588,7 @@ Test Cases
    that checks if clearAttributes() was called correctly.
    (Uwe Schindler, Robert Muir)
 
- * LUCENE-2207, LUCENE-2219: Improve BaseTokenStreamTestCase to check if 
+ * LUCENE-2207, LUCENE-2219: Improve BaseTokenStreamTestCase to check if
    end() is implemented correctly.  (Koji Sekiguchi, Robert Muir)
 
 Documentation
@@ -13594,7 +13596,7 @@ Documentation
  * LUCENE-2114: Improve javadocs of Filter to call out that the
    provided reader is per-segment (Simon Willnauer via Mike
    McCandless)
- 
+
 ======================= Release 3.0.0 =======================
 
 Changes in backwards compatibility policy
@@ -13631,7 +13633,7 @@ Changes in backwards compatibility policy
   this is a change in the default from 2.9, when these methods were
   previously deprecated.  (Mike McCandless)
 
-* LUCENE-1753: Make not yet final TokenStreams final to enforce 
+* LUCENE-1753: Make not yet final TokenStreams final to enforce
   decorator pattern. (Uwe Schindler)
 
 Changes in runtime behavior
@@ -13698,7 +13700,7 @@ API Changes
   - o.a.l.util.cache package
   - lot's of internal APIs of IndexWriter
   (Uwe Schindler, Michael Busch, Kay Kay, Robert Muir, Adriano Crestani)
-  
+
 * LUCENE-1944, LUCENE-1856, LUCENE-1957, LUCENE-1960, LUCENE-1961,
   LUCENE-1968, LUCENE-1970, LUCENE-1946, LUCENE-1971, LUCENE-1975,
   LUCENE-1972, LUCENE-1978, LUCENE-944, LUCENE-1979, LUCENE-1973, LUCENE-2011:
@@ -13729,7 +13731,7 @@ API Changes
   - Remove MultiValueSource.
   - Remove Scorer.explain(int).
   ...and some other minor ones (Uwe Schindler, Michael Busch, Mark Miller)
-  
+
 * LUCENE-1925: Make IndexSearcher's subReaders and docStarts members
   protected; add expert ctor to directly specify reader, subReaders
   and docStarts.  (John Wang, Tim Smith via Mike McCandless)
@@ -13742,7 +13744,7 @@ API Changes
   is no backwards-break, only a change of the super class. Parameter
   was deprecated and will be removed in a later version.
   (DM Smith, Uwe Schindler)
-   
+
 Bug fixes
 
 * LUCENE-1951: When the text provided to WildcardQuery has no wildcard
@@ -13773,7 +13775,7 @@ New features
 * LUCENE-2041: Parallelize the rest of ParallelMultiSearcher. Lots of
   code refactoring and Java 5 concurrent support in MultiSearcher.
   (Joey Surls, Simon Willnauer via Uwe Schindler)
-  
+
 * LUCENE-2051: Add CharArraySet.copy() as a simple method to copy
   any Set<?> to a CharArraySet that is optimized, if Set<?> is already
   an CharArraySet.  (Simon Willnauer)
@@ -13854,13 +13856,13 @@ API Changes
 
  * LUCENE-2043: Make IndexReader.commit(Map<String,String>) public.
    (Mike McCandless)
-   
+
 Documentation
 
  * LUCENE-1955: Fix Hits deprecation notice to point users in right
    direction. (Mike McCandless, Mark Miller)
-   
- * Fix javadoc about score tracking done by search methods in Searcher 
+
+ * Fix javadoc about score tracking done by search methods in Searcher
    and IndexSearcher.  (Mike McCandless)
 
  * LUCENE-2008: Javadoc improvements for TokenStream/Tokenizer/Token
@@ -13880,9 +13882,9 @@ Changes in backwards compatibility policy
     Alternatively, use Searchable.search(Weight, Filter, Collector)
     and pass in a TopFieldCollector instance, using the following code
     sample:
- 
+
     <code>
-      TopFieldCollector tfc = TopFieldCollector.create(sort, numHits, fillFields, 
+      TopFieldCollector tfc = TopFieldCollector.create(sort, numHits, fillFields,
                                                        true /* trackDocScores */,
                                                        true /* trackMaxScore */,
                                                        false /* docsInOrder */);
@@ -13899,7 +13901,7 @@ Changes in backwards compatibility policy
     implement Searchable or extend Searcher, you should change your
     code to implement this method.  If you already extend
     IndexSearcher, no further changes are needed to use Collector.
-    
+
     Finally, the values Float.NaN and Float.NEGATIVE_INFINITY are not
     valid scores.  Lucene uses these values internally in certain
     places, so if you have hits with such scores, it will cause
@@ -13907,35 +13909,35 @@ Changes in backwards compatibility policy
 
  * LUCENE-1687: All methods and parsers from the interface ExtendedFieldCache
     have been moved into FieldCache. ExtendedFieldCache is now deprecated and
-    contains only a few declarations for binary backwards compatibility. 
-    ExtendedFieldCache will be removed in version 3.0. Users of FieldCache and 
+    contains only a few declarations for binary backwards compatibility.
+    ExtendedFieldCache will be removed in version 3.0. Users of FieldCache and
     ExtendedFieldCache will be able to plug in Lucene 2.9 without recompilation.
     The auto cache (FieldCache.getAuto) is now deprecated. Due to the merge of
     ExtendedFieldCache and FieldCache, FieldCache can now additionally return
     long[] and double[] arrays in addition to int[] and float[] and StringIndex.
-    
+
     The interface changes are only notable for users implementing the interfaces,
     which was unlikely done, because there is no possibility to change
     Lucene's FieldCache implementation.  (Grant Ingersoll, Uwe Schindler)
-    
- * LUCENE-1630, LUCENE-1771: Weight, previously an interface, is now an abstract 
+
+ * LUCENE-1630, LUCENE-1771: Weight, previously an interface, is now an abstract
     class. Some of the method signatures have changed, but it should be fairly
     easy to see what adjustments must be made to existing code to sync up
     with the new API. You can find more detail in the API Changes section.
-    
+
     Going forward Searchable will be kept for convenience only and may
     be changed between minor releases without any deprecation
     process. It is not recommended that you implement it, but rather extend
-    Searcher.  
+    Searcher.
     (Shai Erera, Chris Hostetter, Martin Ruckli, Mark Miller via Mike McCandless)
 
  * LUCENE-1422, LUCENE-1693: The new Attribute based TokenStream API (see below)
-    has some backwards breaks in rare cases. We did our best to make the 
-    transition as easy as possible and you are not likely to run into any problems. 
-    If your tokenizers still implement next(Token) or next(), the calls are 
-    automatically wrapped. The indexer and query parser use the new API 
-    (eg use incrementToken() calls). All core TokenStreams are implemented using 
-    the new API. You can mix old and new API style TokenFilters/TokenStream. 
+    has some backwards breaks in rare cases. We did our best to make the
+    transition as easy as possible and you are not likely to run into any problems.
+    If your tokenizers still implement next(Token) or next(), the calls are
+    automatically wrapped. The indexer and query parser use the new API
+    (eg use incrementToken() calls). All core TokenStreams are implemented using
+    the new API. You can mix old and new API style TokenFilters/TokenStream.
     Problems only occur when you have done the following:
     You have overridden next(Token) or next() in one of the non-abstract core
     TokenStreams/-Filters. These classes should normally be final, but some
@@ -13948,28 +13950,28 @@ Changes in backwards compatibility policy
     be passed upon instantiation. As a result, IndexWriter was removed
     as a method argument from all MergePolicy methods. (Shai Erera via
     Mike McCandless)
-    
+
  * LUCENE-1748: LUCENE-1001 introduced PayloadSpans, but this was a back
     compat break and caused custom SpanQuery implementations to fail at runtime
     in a variety of ways. This issue attempts to remedy things by causing
-    a compile time break on custom SpanQuery implementations and removing 
+    a compile time break on custom SpanQuery implementations and removing
     the PayloadSpans class, with its functionality now moved to Spans. To
     help in alleviating future back compat pain, Spans has been changed from
     an interface to an abstract class.
     (Hugh Cayless, Mark Miller)
-    
+
  * LUCENE-1808: Query.createWeight has been changed from protected to
     public. This will be a back compat break if you have overridden this
-    method - but you are likely already affected by the LUCENE-1693 (make Weight 
-    abstract rather than an interface) back compat break if you have overridden 
+    method - but you are likely already affected by the LUCENE-1693 (make Weight
+    abstract rather than an interface) back compat break if you have overridden
     Query.creatWeight, so we have taken the opportunity to make this change.
     (Tim Smith, Shai Erera via Mark Miller)
 
  * LUCENE-1708 - IndexReader.document() no longer checks if the document is
     deleted. You can call IndexReader.isDeleted(n) prior to calling document(n).
     (Shai Erera via Mike McCandless)
-    
- 
+
+
 Changes in runtime behavior
 
  * LUCENE-1424: QueryParser now by default uses constant score auto
@@ -13978,7 +13980,7 @@ Changes in runtime behavior
     setMultiTermRewriteMethod(MultiTermQuery.SCORING_BOOLEAN_QUERY_REWRITE)
     to revert to slower BooleanQuery rewriting method.  (Mark Miller via Mike
     McCandless)
-    
+
  * LUCENE-1575: As of 2.9, the core collectors as well as
     IndexSearcher's search methods that return top N results, no
     longer filter documents with scores <= 0.0. If you rely on this
@@ -14045,7 +14047,7 @@ Changes in runtime behavior
     reusableTokenStream.  This is now fixed, such that if
     reusableTokenStream is invoked on such a subclass, that method
     will forcefully fallback to tokenStream.  (Mike McCandless)
-    
+
  * LUCENE-1801: Token.clear() and Token.clearNoTermBuffer() now also clear
     startOffset, endOffset and type. This is not likely to affect any
     Tokenizer chains, as Tokenizers normally always set these three values.
@@ -14053,17 +14055,17 @@ Changes in runtime behavior
     AttributeSource.clearAttributes() to work identical for Token as one for all
     AttributeImpl and the 6 separate AttributeImpls. (Uwe Schindler, Michael Busch)
 
- * LUCENE-1483: When searching over multiple segments, a new Scorer is now created 
+ * LUCENE-1483: When searching over multiple segments, a new Scorer is now created
     for each segment. Searching has been telescoped out a level and IndexSearcher now
-    operates much like MultiSearcher does. The Weight is created only once for the top 
-    level Searcher, but each Scorer is passed a per-segment IndexReader. This will 
-    result in doc ids in the Scorer being internal to the per-segment IndexReader. It 
-    has always been outside of the API to count on a given IndexReader to contain every 
-    doc id in the index - and if you have been ignoring MultiSearcher in your custom code 
-    and counting on this fact, you will find your code no longer works correctly. If a 
-    custom Scorer implementation uses any caches/filters that rely on being based on the 
-    top level IndexReader, it will need to be updated to correctly use contextless 
-    caches/filters eg you can't count on the IndexReader to contain any given doc id or 
+    operates much like MultiSearcher does. The Weight is created only once for the top
+    level Searcher, but each Scorer is passed a per-segment IndexReader. This will
+    result in doc ids in the Scorer being internal to the per-segment IndexReader. It
+    has always been outside of the API to count on a given IndexReader to contain every
+    doc id in the index - and if you have been ignoring MultiSearcher in your custom code
+    and counting on this fact, you will find your code no longer works correctly. If a
+    custom Scorer implementation uses any caches/filters that rely on being based on the
+    top level IndexReader, it will need to be updated to correctly use contextless
+    caches/filters eg you can't count on the IndexReader to contain any given doc id or
     all of the doc ids. (Mark Miller, Mike McCandless)
 
  * LUCENE-1846: DateTools now uses the US locale to format the numbers in its
@@ -14080,7 +14082,7 @@ Changes in runtime behavior
 
 API Changes
 
- * LUCENE-1419: Add expert API to set custom indexing chain. This API is 
+ * LUCENE-1419: Add expert API to set custom indexing chain. This API is
    package-protected for now, so we don't have to officially support it.
    Yet, it will give us the possibility to try out different consumers
    in the chain. (Michael Busch)
@@ -14088,20 +14090,20 @@ API Changes
  * LUCENE-1427: DocIdSet.iterator() is now allowed to throw
    IOException.  (Paul Elschot, Mike McCandless)
 
- * LUCENE-1422, LUCENE-1693: New TokenStream API that uses a new class called 
+ * LUCENE-1422, LUCENE-1693: New TokenStream API that uses a new class called
    AttributeSource instead of the Token class, which is now a utility class that
-   holds common Token attributes. All attributes that the Token class had have 
-   been moved into separate classes: TermAttribute, OffsetAttribute, 
-   PositionIncrementAttribute, PayloadAttribute, TypeAttribute and FlagsAttribute. 
-   The new API is much more flexible; it allows to combine the Attributes 
-   arbitrarily and also to define custom Attributes. The new API has the same 
-   performance as the old next(Token) approach. For conformance with this new 
-   API Tee-/SinkTokenizer was deprecated and replaced by a new TeeSinkTokenFilter. 
-   (Michael Busch, Uwe Schindler; additional contributions and bug fixes by 
+   holds common Token attributes. All attributes that the Token class had have
+   been moved into separate classes: TermAttribute, OffsetAttribute,
+   PositionIncrementAttribute, PayloadAttribute, TypeAttribute and FlagsAttribute.
+   The new API is much more flexible; it allows to combine the Attributes
+   arbitrarily and also to define custom Attributes. The new API has the same
+   performance as the old next(Token) approach. For conformance with this new
+   API Tee-/SinkTokenizer was deprecated and replaced by a new TeeSinkTokenFilter.
+   (Michael Busch, Uwe Schindler; additional contributions and bug fixes by
    Daniel Shane, Doron Cohen)
 
  * LUCENE-1467: Add nextDoc() and next(int) methods to OpenBitSetIterator.
-   These methods can be used to avoid additional calls to doc(). 
+   These methods can be used to avoid additional calls to doc().
    (Michael Busch)
 
  * LUCENE-1468: Deprecate Directory.list(), which sometimes (in
@@ -14123,12 +14125,12 @@ API Changes
 
  * LUCENE-1561: Renamed Field.omitTf to Field.omitTermFreqAndPositions
     (Otis Gospodnetic via Mike McCandless)
-  
+
  * LUCENE-1500: Added new InvalidTokenOffsetsException to Highlighter methods
     to denote issues when offsets in TokenStream tokens exceed the length of the
     provided text.  (Mark Harwood)
-    
- * LUCENE-1575, LUCENE-1483: HitCollector is now deprecated in favor of 
+
+ * LUCENE-1575, LUCENE-1483: HitCollector is now deprecated in favor of
     a new Collector abstract class. For easy migration, people can use
     HitCollectorWrapper which translates (wraps) HitCollector into
     Collector. Note that this class is also deprecated and will be
@@ -14166,14 +14168,14 @@ API Changes
     McCandless)
 
  * LUCENE-1614: DocIdSetIterator's next() and skipTo() were deprecated in favor
-    of the new nextDoc() and advance(). The new methods return the doc Id they 
+    of the new nextDoc() and advance(). The new methods return the doc Id they
     landed on, saving an extra call to doc() in most cases.
-    For easy migration of the code, you can change the calls to next() to 
-    nextDoc() != DocIdSetIterator.NO_MORE_DOCS and similarly for skipTo(). 
-    However it is advised that you take advantage of the returned doc ID and not 
+    For easy migration of the code, you can change the calls to next() to
+    nextDoc() != DocIdSetIterator.NO_MORE_DOCS and similarly for skipTo().
+    However it is advised that you take advantage of the returned doc ID and not
     call doc() following those two.
-    Also, doc() was deprecated in favor of docID(). docID() should return -1 or 
-    NO_MORE_DOCS if nextDoc/advance were not called yet, or NO_MORE_DOCS if the 
+    Also, doc() was deprecated in favor of docID(). docID() should return -1 or
+    NO_MORE_DOCS if nextDoc/advance were not called yet, or NO_MORE_DOCS if the
     iterator has exhausted. Otherwise it should return the current doc ID.
     (Shai Erera via Mike McCandless)
 
@@ -14197,25 +14199,25 @@ API Changes
  * LUCENE-1673: Deprecated NumberTools in favour of the new
     NumericRangeQuery and its new indexing format for numeric or
     date values.  (Uwe Schindler)
-    
+
  * LUCENE-1630, LUCENE-1771: Weight is now an abstract class, and adds
     a scorer(IndexReader, boolean /* scoreDocsInOrder */, boolean /*
-    topScorer */) method instead of scorer(IndexReader). IndexSearcher uses 
-    this method to obtain a scorer matching the capabilities of the Collector 
+    topScorer */) method instead of scorer(IndexReader). IndexSearcher uses
+    this method to obtain a scorer matching the capabilities of the Collector
     wrt orderedness of docIDs. Some Scorers (like BooleanScorer) are much more
-    efficient if out-of-order documents scoring is allowed by a Collector.  
-    Collector must now implement acceptsDocsOutOfOrder. If you write a 
-    Collector which does not care about doc ID orderness, it is recommended 
-    that you return true.  Weight has a scoresDocsOutOfOrder method, which by 
-    default returns false.  If you create a Weight which will score documents 
-    out of order if requested, you should override that method to return true. 
-    BooleanQuery's setAllowDocsOutOfOrder and getAllowDocsOutOfOrder have been 
-    deprecated as they are not needed anymore. BooleanQuery will now score docs 
+    efficient if out-of-order documents scoring is allowed by a Collector.
+    Collector must now implement acceptsDocsOutOfOrder. If you write a
+    Collector which does not care about doc ID orderness, it is recommended
+    that you return true.  Weight has a scoresDocsOutOfOrder method, which by
+    default returns false.  If you create a Weight which will score documents
+    out of order if requested, you should override that method to return true.
+    BooleanQuery's setAllowDocsOutOfOrder and getAllowDocsOutOfOrder have been
+    deprecated as they are not needed anymore. BooleanQuery will now score docs
     out of order when used with a Collector that can accept docs out of order.
     Finally, Weight#explain now takes a sub-reader and sub-docID, rather than
     a top level reader and docID.
     (Shai Erera, Chris Hostetter, Martin Ruckli, Mark Miller via Mike McCandless)
-   
+
  * LUCENE-1466, LUCENE-1906: Added CharFilter and MappingCharFilter, which allows
     chaining & mapping of characters before tokenizers run. CharStream (subclass of
     Reader) is the base class for custom java.io.Reader's, that support offset
@@ -14244,16 +14246,16 @@ API Changes
     all synchronization in TermInfosReader, which previously could
     cause threads to pile up in certain cases. (Dan Rosher via Mike
     McCandless)
-    
- * LUCENE-1688: Deprecate static final String stop word array in and 
-    StopAnalzyer and replace it with an immutable implementation of 
+
+ * LUCENE-1688: Deprecate static final String stop word array in and
+    StopAnalzyer and replace it with an immutable implementation of
     CharArraySet.  (Simon Willnauer via Mark Miller)
 
  * LUCENE-1742: SegmentInfos, SegmentInfo and SegmentReader have been
     made public as expert, experimental APIs.  These APIs may suddenly
     change from release to release (Jason Rutherglen via Mike
     McCandless).
-    
+
  * LUCENE-1754: QueryWeight.scorer() can return null if no documents
     are going to be matched by the query. Similarly,
     Filter.getDocIdSet() can return null if no documents are going to
@@ -14273,23 +14275,23 @@ API Changes
  * LUCENE-1748: LUCENE-1001 introduced PayloadSpans, but this was a back
     compat break and caused custom SpanQuery implementations to fail at runtime
     in a variety of ways. This issue attempts to remedy things by causing
-    a compile time break on custom SpanQuery implementations and removing 
+    a compile time break on custom SpanQuery implementations and removing
     the PayloadSpans class, with its functionality now moved to Spans. To
     help in alleviating future back compat pain, Spans has been changed from
     an interface to an abstract class.
     (Hugh Cayless, Mark Miller)
-    
+
  * LUCENE-1808: Query.createWeight has been changed from protected to
     public. (Tim Smith, Shai Erera via Mark Miller)
 
  * LUCENE-1826: Add constructors that take AttributeSource and
     AttributeFactory to all Tokenizer implementations.
     (Michael Busch)
-    
+
  * LUCENE-1847: Similarity#idf for both a Term and Term Collection have
     been deprecated. New versions that return an IDFExplanation have been
     added.  (Yasoja Seneviratne, Mike McCandless, Mark Miller)
-    
+
  * LUCENE-1877: Made NativeFSLockFactory the default for
     the new FSDirectory API (open(), FSDirectory subclass ctors).
     All FSDirectory system properties were deprecated and all lock
@@ -14311,7 +14313,7 @@ API Changes
 Bug fixes
 
  * LUCENE-1415: MultiPhraseQuery has incorrect hashCode() and equals()
-   implementation - Leads to Solr Cache misses. 
+   implementation - Leads to Solr Cache misses.
    (Todd Feak, Mark Miller via yonik)
 
  * LUCENE-1327: Fix TermSpans#skipTo() to behave as specified in javadocs
@@ -14331,7 +14333,7 @@ Bug fixes
 
  * LUCENE-1587: RangeQuery#equals() could consider a RangeQuery
    without a collator equal to one with a collator.
-   (Mark Platvoet via Mark Miller) 
+   (Mark Platvoet via Mark Miller)
 
  * LUCENE-1600: Don't call String.intern unnecessarily in some cases
    when loading documents from the index.  (P Eger via Mike
@@ -14345,8 +14347,8 @@ Bug fixes
    contain field names with non-ascii characters.  (Mike Streeton via
    Mike McCandless)
 
- * LUCENE-1593: MultiSearcher and ParallelMultiSearcher did not break ties (in 
-   sort) by doc Id in a consistent manner (i.e., if Sort.FIELD_DOC was used vs. 
+ * LUCENE-1593: MultiSearcher and ParallelMultiSearcher did not break ties (in
+   sort) by doc Id in a consistent manner (i.e., if Sort.FIELD_DOC was used vs.
    when it wasn't). (Shai Erera via Michael McCandless)
 
  * LUCENE-1647: Fix case where IndexReader.undeleteAll would cause
@@ -14366,8 +14368,8 @@ Bug fixes
     on EOF, removed numeric overflow possibilities and added support
     for a hack to unmap the buffers on closing IndexInput.
     (Uwe Schindler)
-    
- * LUCENE-1681: Fix infinite loop caused by a call to DocValues methods 
+
+ * LUCENE-1681: Fix infinite loop caused by a call to DocValues methods
     getMinValue, getMaxValue, getAverageValue. (Simon Willnauer via Mark Miller)
 
  * LUCENE-1599: Add clone support for SpanQuerys. SpanRegexQuery counts
@@ -14376,7 +14378,7 @@ Bug fixes
 
  * LUCENE-1718: Fix termInfosIndexDivisor to carry over to reopened
     readers (Mike McCandless)
-    
+
  * LUCENE-1583: SpanOrQuery skipTo() doesn't always move forwards as Spans
   documentation indicates it should.  (Moti Nisenson via Mark Miller)
 
@@ -14389,30 +14391,30 @@ Bug fixes
     JVMs the default chunk size is 100 MB; on 64bit JVMs, which don't
     show the bug, the default is Integer.MAX_VALUE. (Simon Willnauer
     via Mike McCandless)
-    
+
  * LUCENE-1448: Added TokenStream.end() to perform end-of-stream
-    operations (ie to return the end offset of the tokenization).  
+    operations (ie to return the end offset of the tokenization).
     This is important when multiple fields with the same name are added
-    to a document, to ensure offsets recorded in term vectors for all 
-    of the instances are correct.  
+    to a document, to ensure offsets recorded in term vectors for all
+    of the instances are correct.
     (Mike McCandless, Mark Miller, Michael Busch)
 
- * LUCENE-1805: CloseableThreadLocal did not allow a null Object in get(), 
+ * LUCENE-1805: CloseableThreadLocal did not allow a null Object in get(),
     although it does allow it in set(Object). Fix get() to not assert the object
     is not null. (Shai Erera via Mike McCandless)
-    
+
  * LUCENE-1801: Changed all Tokenizers or TokenStreams in core/contrib)
     that are the source of Tokens to always call
     AttributeSource.clearAttributes() first. (Uwe Schindler)
-    
+
  * LUCENE-1819: MatchAllDocsQuery.toString(field) should produce output
     that is parsable by the QueryParser.  (John Wang, Mark Miller)
 
- * LUCENE-1836: Fix localization bug in the new query parser and add 
+ * LUCENE-1836: Fix localization bug in the new query parser and add
     new LocalizedTestCase as base class for localization junit tests.
     (Robert Muir, Uwe Schindler via Michael Busch)
 
- * LUCENE-1847: PhraseQuery/TermQuery/SpanQuery use IndexReader specific stats 
+ * LUCENE-1847: PhraseQuery/TermQuery/SpanQuery use IndexReader specific stats
     in their Weight#explain methods - these stats should be corpus wide.
     (Yasoja Seneviratne, Mike McCandless, Mark Miller)
 
@@ -14476,13 +14478,13 @@ New features
     support collation  (Tim Sturge, Matt Ericson via Mike McCandless and
     Uwe Schindler)
 
- * LUCENE-1296: add protected method CachingWrapperFilter.docIdSetToCache 
+ * LUCENE-1296: add protected method CachingWrapperFilter.docIdSetToCache
     to allow subclasses to choose which DocIdSet implementation to use
     (Paul Elschot via Mike McCandless)
-    
- * LUCENE-1390: Added ASCIIFoldingFilter, a Filter that converts 
-    alphabetic, numeric, and symbolic Unicode characters which are not in 
-    the first 127 ASCII characters (the "Basic Latin" Unicode block) into 
+
+ * LUCENE-1390: Added ASCIIFoldingFilter, a Filter that converts
+    alphabetic, numeric, and symbolic Unicode characters which are not in
+    the first 127 ASCII characters (the "Basic Latin" Unicode block) into
     their ASCII equivalents, if one exists. ISOLatin1AccentFilter, which
     handles a subset of this filter, has been deprecated.
     (Andi Vajda, Steven Rowe via Mark Miller)
@@ -14550,7 +14552,7 @@ New features
 
  * LUCENE-1605: Added BitVector.subset().  (Jeremy Volkman via Mike
     McCandless)
-    
+
  * LUCENE-1618: Added FileSwitchDirectory that enables files with
     specified extensions to be stored in a primary directory and the
     rest of the files to be stored in the secondary directory.  For
@@ -14589,12 +14591,12 @@ New features
  * LUCENE-1699: Allow setting a TokenStream on Field/Fieldable for indexing
     in conjunction with any other ways to specify stored field values,
     currently binary or string values.  (yonik)
-    
+
  * LUCENE-1701: Made the standard FieldCache.Parsers public and added
     parsers for fields generated using NumericField/NumericTokenStream.
     All standard parsers now also implement Serializable and enforce
     their singleton status.  (Uwe Schindler, Mike McCandless)
-    
+
  * LUCENE-1741: User configurable maximum chunk size in MMapDirectory.
     On 32 bit platforms, the address space can be very fragmented, so
     one big ByteBuffer for the whole file may not fit into address space.
@@ -14609,12 +14611,12 @@ New features
     creates a BooleanQuery but keeps the BooleanQuery's scores;
     CONSTANT_SCORE_AUTO_REWRITE tries to pick the most performant
     constant-score rewrite method.  (Mike McCandless)
-    
+
  * LUCENE-1448: Added TokenStream.end(), to perform end-of-stream
-    operations.  This is currently used to fix offset problems when 
+    operations.  This is currently used to fix offset problems when
     multiple fields with the same name are added to a document.
     (Mike McCandless, Mark Miller, Michael Busch)
- 
+
  * LUCENE-1776: Add an option to not collect payloads for an ordered
     SpanNearQuery. Payloads were not lazily loaded in this case as
     the javadocs implied. If you have payloads and want to use an ordered
@@ -14635,7 +14637,7 @@ New features
     FieldCache usage.  Namely: FieldCache entries for the same field
     with different datatypes or parsers; and FieldCache entries for
     the same field in both a reader, and one of its (descendant) sub
-    readers. 
+    readers.
     (Chris Hostetter, Mark Miller)
 
  * LUCENE-1789: Added utility class
@@ -14645,7 +14647,7 @@ New features
     ValueSource, but takes care when composite (multi-segment) are
     passed to not double RAM usage in the FieldCache.  (Chris
     Hostetter, Mark Miller, Mike McCandless)
-    
+
 Optimizations
 
  * LUCENE-1427: Fixed QueryWrapperFilter to not waste time computing
@@ -14659,8 +14661,8 @@ Optimizations
  * LUCENE-1484: Remove synchronization of IndexReader.document() by
     using CloseableThreadLocal internally.  (Jason Rutherglen via Mike
     McCandless).
-    
- * LUCENE-1124: Short circuit FuzzyQuery.rewrite when input token length 
+
+ * LUCENE-1124: Short circuit FuzzyQuery.rewrite when input token length
     is small compared to minSimilarity. (Timo Nentwig, Mark Miller)
 
  * LUCENE-1316: MatchAllDocsQuery now avoids the synchronized
@@ -14676,7 +14678,7 @@ Optimizations
     those segments that did not change, and also speeds up searches
     that sort by relevance or by field values.  (Mark Miller, Mike
     McCandless)
-    
+
  * LUCENE-1575: The new Collector class decouples collect() from
     score computation.  Collector.setScorer is called to establish the
     current Scorer in-use per segment.  Collectors that require the
@@ -14685,12 +14687,12 @@ Optimizations
 
  * LUCENE-1596: MultiTermDocs speedup when set with
     MultiTermDocs.seek(MultiTermEnum) (yonik)
-    
- * LUCENE-1653: Avoid creating a Calendar in every call to 
+
+ * LUCENE-1653: Avoid creating a Calendar in every call to
     DateTools#dateToString, DateTools#timeToString and
     DateTools#round.  (Shai Erera via Mark Miller)
-    
- * LUCENE-1688: Deprecate static final String stop word array and 
+
+ * LUCENE-1688: Deprecate static final String stop word array and
     replace it with an immutable implementation of CharArraySet.
     Removes conversions between Set and array.
     (Simon Willnauer via Mark Miller)
@@ -14706,16 +14708,16 @@ Optimizations
     (Earwin Burrfoot, yonik)
 
  * LUCENE-1800: QueryParser should use reusable TokenStreams. (yonik)
-    
+
 
 Documentation
 
- * LUCENE-1908: Scoring documentation imrovements in Similarity javadocs. 
+ * LUCENE-1908: Scoring documentation imrovements in Similarity javadocs.
    (Mark Miller, Shai Erera, Ted Dunning, Jiri Kuhn, Marvin Humphrey, Doron Cohen)
-    
+
  * LUCENE-1872: NumericField javadoc improvements
     (Michael McCandless, Uwe Schindler)
- 
+
  * LUCENE-1875: Make TokenStream.end javadoc less confusing.
     (Uwe Schindler)
 
@@ -14729,14 +14731,14 @@ Documentation
  * LUCENE-1884: massive javadoc and comment cleanup, primarily dealing with
     typos.
     (Robert Muir via Chris Hostetter)
-    
- * LUCENE-1898: Switch changes to use bullets rather than numbers and 
-    update changes-to-html script to handle the new format. 
+
+ * LUCENE-1898: Switch changes to use bullets rather than numbers and
+    update changes-to-html script to handle the new format.
     (Steven Rowe, Mark Miller)
-    
+
  * LUCENE-1900: Improve Searchable Javadoc.
     (Nadav Har'El, Doron Cohen, Marvin Humphrey, Mark Miller)
-    
+
  * LUCENE-1896: Improve Similarity#queryNorm javadocs.
     (Jiri Kuhn, Mark Miller)
 
@@ -14746,10 +14748,10 @@ Build
     and executing the junit testcases from an older release for
     backwards-compatibility testing. (Michael Busch)
 
- * LUCENE-1446: Add compatibility tag to common-build.xml and run 
+ * LUCENE-1446: Add compatibility tag to common-build.xml and run
     backwards-compatibility tests in the nightly build. (Michael Busch)
 
- * LUCENE-1529: Properly test "drop-in" replacement of jar with 
+ * LUCENE-1529: Properly test "drop-in" replacement of jar with
     backwards-compatibility tests. (Mike McCandless, Michael Busch)
 
  * LUCENE-1851: Change 'javacc' and 'clean-javacc' targets to build
@@ -14760,14 +14762,14 @@ Build
 
 Test Cases
 
- * LUCENE-1791: Enhancements to the QueryUtils and CheckHits utility 
-    classes to wrap IndexReaders and Searchers in MultiReaders or 
+ * LUCENE-1791: Enhancements to the QueryUtils and CheckHits utility
+    classes to wrap IndexReaders and Searchers in MultiReaders or
     MultiSearcher when possible to help exercise more edge cases.
     (Chris Hostetter, Mark Miller)
 
- * LUCENE-1852: Fix localization test failures. 
+ * LUCENE-1852: Fix localization test failures.
     (Robert Muir via Michael Busch)
-    
+
  * LUCENE-1843: Refactored all tests that use assertAnalyzesTo() & others
     in core and contrib to use a new BaseTokenStreamTestCase
     base class. Also rewrote some tests to use this general analysis assert
@@ -14776,7 +14778,7 @@ Test Cases
     backwards layer enabled (using Token/TokenWrapper as attribute
     implementation) and disabled (default for Lucene 3.0)
     (Uwe Schindler, Robert Muir)
-    
+
  * LUCENE-1836: Added a new LocalizedTestCase as base class for localization
     junit tests.  (Robert Muir, Uwe Schindler via Michael Busch)
 
@@ -14814,7 +14816,7 @@ Bug fixes
    incorrectly closing the shared FSDirectory. This bug would only
    happen if you use IndexReader.open() with a File or String argument.
    The returned readers are wrapped by a FilterIndexReader that
-   correctly handles closing of directory after reopen()/clone(). 
+   correctly handles closing of directory after reopen()/clone().
    (Mark Miller, Uwe Schindler, Mike McCandless)
 
 7. LUCENE-1457: Fix possible overflow bugs during binary
@@ -14836,9 +14838,9 @@ Bug fixes
 12. LUCENE-1547: Fix rare thread safety issue if two threads call
     IndexWriter commit() at the same time.  (Mike McCandless)
 
-13. LUCENE-1465: NearSpansOrdered returns payloads from first possible match 
+13. LUCENE-1465: NearSpansOrdered returns payloads from first possible match
     rather than the correct, shortest match; Payloads could be returned even
-    if the max slop was exceeded; The wrong payload could be returned in 
+    if the max slop was exceeded; The wrong payload could be returned in
     certain situations. (Jonathan Mamou, Greg Shackles, Mark Miller)
 
 14. LUCENE-1186: Add Analyzer.close() to free internal ThreadLocal
@@ -14937,7 +14939,7 @@ API Changes
 
  8. LUCENE-852: Let the SpellChecker caller specify IndexWriter mergeFactor
     and RAM buffer size.  (Otis Gospodnetic)
-  
+
  9. LUCENE-1290: Deprecate org.apache.lucene.search.Hits, Hit and HitIterator
     and remove all references to these classes from the core. Also update demos
     and tutorials. (Michael Busch)
@@ -15004,10 +15006,10 @@ API Changes
 
 22. LUCENE-1371: Added convenience method TopDocs Searcher.search(Query query, int n).
     (Mike McCandless)
-    
+
 23. LUCENE-1356: Allow easy extensions of TopDocCollector by turning
     constructor and fields from package to protected. (Shai Erera
-    via Doron Cohen) 
+    via Doron Cohen)
 
 24. LUCENE-1375: Added convenience method IndexCommit.getTimestamp,
     which is equivalent to
@@ -15021,18 +15023,18 @@ API Changes
 24. LUCENE-1131: Added numDeletedDocs method to IndexReader (Otis Gospodnetic)
 
 Bug fixes
-    
- 1. LUCENE-1134: Fixed BooleanQuery.rewrite to only optimize a single 
+
+ 1. LUCENE-1134: Fixed BooleanQuery.rewrite to only optimize a single
     clause query if minNumShouldMatch<=0. (Shai Erera via Michael Busch)
 
  2. LUCENE-1169: Fixed bug in IndexSearcher.search(): searching with
     a filter might miss some hits because scorer.skipTo() is called
     without checking if the scorer is already at the right position.
-    scorer.skipTo(scorer.doc()) is not a NOOP, it behaves as 
+    scorer.skipTo(scorer.doc()) is not a NOOP, it behaves as
     scorer.next(). (Eks Dev, Michael Busch)
 
  3. LUCENE-1182: Added scorePayload to SimilarityDelegator (Andi Vajda via Grant Ingersoll)
- 
+
  4. LUCENE-1213: MultiFieldQueryParser was ignoring slop in case
     of a single field phrase. (Trejkaz via Doron Cohen)
 
@@ -15057,18 +15059,18 @@ Bug fixes
 
 10. LUCENE-1046: Removed dead code in SpellChecker
     (Daniel Naber via Otis Gospodnetic)
-  
-11. LUCENE-1189: Fixed the QueryParser to handle escaped characters within 
+
+11. LUCENE-1189: Fixed the QueryParser to handle escaped characters within
     quoted terms correctly. (Tomer Gabel via Michael Busch)
 
 12. LUCENE-1299: Fixed NPE in SpellChecker when IndexReader is not null and field is (Grant Ingersoll)
 
-13. LUCENE-1303: Fixed BoostingTermQuery's explanation to be marked as a Match 
-    depending only upon the non-payload score part, regardless of the effect of 
-    the payload on the score. Prior to this, score of a query containing a BTQ 
+13. LUCENE-1303: Fixed BoostingTermQuery's explanation to be marked as a Match
+    depending only upon the non-payload score part, regardless of the effect of
+    the payload on the score. Prior to this, score of a query containing a BTQ
     differed from its explanation. (Doron Cohen)
-    
-14. LUCENE-1310: Fixed SloppyPhraseScorer to work also for terms repeating more 
+
+14. LUCENE-1310: Fixed SloppyPhraseScorer to work also for terms repeating more
     than twice in the query. (Doron Cohen)
 
 15. LUCENE-1351: ISOLatin1AccentFilter now cleans additional ligatures (Cedrik Lime via Grant Ingersoll)
@@ -15087,10 +15089,10 @@ New features
     check only a specific segment or segments in your index.  (Mike
     McCandless)
 
- 3. LUCENE-1045: Reopened this issue to add support for short and bytes. 
- 
- 4. LUCENE-584: Added new data structures to o.a.l.util, such as 
-    OpenBitSet and SortedVIntList. These extend DocIdSet and can 
+ 3. LUCENE-1045: Reopened this issue to add support for short and bytes.
+
+ 4. LUCENE-584: Added new data structures to o.a.l.util, such as
+    OpenBitSet and SortedVIntList. These extend DocIdSet and can
     directly be used for Filters with the new Filter API. Also changed
     the core Filters to use OpenBitSet instead of java.util.BitSet.
     (Paul Elschot, Michael Busch)
@@ -15103,13 +15105,13 @@ New features
     with cached writes, the index remains consistent.  Also added
     explicit commit() method to IndexWriter to force a commit without
     having to close.  (Mike McCandless)
-    
+
  7. LUCENE-997: Add search timeout (partial) support.
     A TimeLimitedCollector was added to allow limiting search time.
-    It is a partial solution since timeout is checked only when 
-    collecting a hit, and therefore a search for rare words in a 
+    It is a partial solution since timeout is checked only when
+    collecting a hit, and therefore a search for rare words in a
     huge index might not stop within the specified time.
-    (Sean Timm via Doron Cohen) 
+    (Sean Timm via Doron Cohen)
 
  8. LUCENE-1184: Allow SnapshotDeletionPolicy to be re-used across
     close/re-open of IndexWriter while still protecting an open
@@ -15123,17 +15125,17 @@ New features
 10. LUCENE-1201: Add IndexReader.getIndexCommit() method. (Tim Brennan
     via Mike McCandless)
 
-11. LUCENE-550:  Added InstantiatedIndex implementation.  Experimental 
-    Index store similar to MemoryIndex but allows for multiple documents 
+11. LUCENE-550:  Added InstantiatedIndex implementation.  Experimental
+    Index store similar to MemoryIndex but allows for multiple documents
     in memory.  (Karl Wettin via Grant Ingersoll)
 
 12. LUCENE-400: Added word based n-gram filter (in contrib/analyzers) called ShingleFilter and an Analyzer wrapper
-    that wraps another Analyzer's token stream with a ShingleFilter (Sebastian Kirsch, Steve Rowe via Grant Ingersoll) 
+    that wraps another Analyzer's token stream with a ShingleFilter (Sebastian Kirsch, Steve Rowe via Grant Ingersoll)
 
 13. LUCENE-1166: Decomposition tokenfilter for languages like German and Swedish (Thomas Peuss via Grant Ingersoll)
 
 14. LUCENE-1187: ChainedFilter and BooleanFilter now work with new Filter API
-    and DocIdSetIterator-based filters. Backwards-compatibility with old 
+    and DocIdSetIterator-based filters. Backwards-compatibility with old
     BitSet-based filters is ensured. (Paul Elschot via Michael Busch)
 
 15. LUCENE-1295: Added new method to MoreLikeThis for retrieving interesting terms and made retrieveTerms(int) public. (Grant Ingersoll)
@@ -15147,7 +15149,7 @@ New features
 
 19. LUCENE-1354: Provide programmatic access to CheckIndex (Grant Ingersoll, Mike McCandless)
 
-20. LUCENE-1279: Add support for Collators to RangeFilter/Query and Query Parser.  (Steve Rowe via Grant Ingersoll) 
+20. LUCENE-1279: Add support for Collators to RangeFilter/Query and Query Parser.  (Steve Rowe via Grant Ingersoll)
 
 Optimizations
 
@@ -15161,8 +15163,8 @@ Optimizations
  2. LUCENE-1120: Speed up merging of term vectors by bulk-copying the
     raw bytes for each contiguous range of non-deleted documents.
     (Mike McCandless)
-  
- 3. LUCENE-1185: Avoid checking if the TermBuffer 'scratch' in 
+
+ 3. LUCENE-1185: Avoid checking if the TermBuffer 'scratch' in
     SegmentTermEnum is null for every call of scanTo().
     (Christian Kohlschuetter via Michael Busch)
 
@@ -15174,45 +15176,45 @@ Optimizations
     less memory than the previous version.  (Cédrik LIME via Otis Gospodnetic)
 
  6. LUCENE-1195: Improve term lookup performance by adding a LRU cache to the
-    TermInfosReader. In performance experiments the speedup was about 25% on 
-    average on mid-size indexes with ~500,000 documents for queries with 3 
+    TermInfosReader. In performance experiments the speedup was about 25% on
+    average on mid-size indexes with ~500,000 documents for queries with 3
     terms and about 7% on larger indexes with ~4.3M documents. (Michael Busch)
 
 Documentation
 
   1. LUCENE-1236:  Added some clarifying remarks to EdgeNGram*.java (Hiroaki Kawai via Grant Ingersoll)
-  
-  2. LUCENE-1157 and LUCENE-1256: HTML changes log, created automatically 
-     from CHANGES.txt. This HTML file is currently visible only via developers page.     
+
+  2. LUCENE-1157 and LUCENE-1256: HTML changes log, created automatically
+     from CHANGES.txt. This HTML file is currently visible only via developers page.
      (Steven Rowe via Doron Cohen)
 
   3. LUCENE-1349: Fieldable can now be changed without breaking backward compatibility rules (within reason.  See the note at
   the top of this file and also on Fieldable.java).  (Grant Ingersoll)
-  
+
   4. LUCENE-1873: Update documentation to reflect current Contrib area status.
      (Steven Rowe, Mark Miller)
 
 Build
 
   1. LUCENE-1153: Added JUnit JAR to new lib directory.  Updated build to rely on local JUnit instead of ANT/lib.
-  
+
   2. LUCENE-1202: Small fixes to the way Clover is used to work better
      with contribs.  Of particular note: a single clover db is used
      regardless of whether tests are run globally or in the specific
-     contrib directories. 
-     
-  3. LUCENE-1353: Javacc target in contrib/miscellaneous for 
-     generating the precedence query parser. 
+     contrib directories.
+
+  3. LUCENE-1353: Javacc target in contrib/miscellaneous for
+     generating the precedence query parser.
 
 Test Cases
 
  1. LUCENE-1238: Fixed intermittent failures of TestTimeLimitedCollector.testTimeoutMultiThreaded.
-    Within this fix, "greedy" flag was added to TimeLimitedCollector, to allow the wrapped 
-    collector to collect also the last doc, after allowed-tTime passed. (Doron Cohen)   
-  
- 2. LUCENE-1348: relax TestTimeLimitedCollector to not fail due to 
+    Within this fix, "greedy" flag was added to TimeLimitedCollector, to allow the wrapped
+    collector to collect also the last doc, after allowed-tTime passed. (Doron Cohen)
+
+ 2. LUCENE-1348: relax TestTimeLimitedCollector to not fail due to
     timeout exceeded (just because test machine is very busy).
-  
+
 ======================= Release 2.3.2 =======================
 
 Bug fixes
@@ -15241,7 +15243,7 @@ Bug fixes
 
  8. LUCENE-1222: Fix IndexWriter.doAfterFlush to always be called on
     flush (Mark Ferguson via Mike McCandless)
-  
+
  9. LUCENE-1226: Fixed IndexWriter.addIndexes(IndexReader[]) to commit
     successfully created compound files. (Michael Busch)
 
@@ -15257,16 +15259,16 @@ Bug fixes
 12. LUCENE-1270: Fixed intermittent case where IndexWriter.close()
     would hang after IndexWriter.addIndexesNoOptimize had been
     called.  (Stu Hood via Mike McCandless)
-  
+
 Build
 
  1. LUCENE-1230: Include *pom.xml* in source release files. (Michael Busch)
 
- 
+
 ======================= Release 2.3.1 =======================
 
 Bug fixes
-    
+
  1. LUCENE-1168: Fixed corruption cases when autoCommit=false and
     documents have mixed term vectors (Suresh Guvvala via Mike
     McCandless).
@@ -15280,14 +15282,14 @@ Bug fixes
  4. LUCENE-1163: Fixed bug in CharArraySet.contains(char[] buffer, int
     offset, int len) that was ignoring offset and thus giving the
     wrong answer.  (Thomas Peuss via Mike McCandless)
-  
+
  5. LUCENE-1177: Fix rare case where IndexWriter.optimize might do too
     many merges at the end.  (Mike McCandless)
-  
+
  6. LUCENE-1176: Fix corruption case when documents with no term
     vector fields are added before documents with term vector fields.
     (Mike McCandless)
-  
+
  7. LUCENE-1179: Fixed assert statement that was incorrectly
     preventing Fields with empty-string field name from working.
     (Sergey Kabashnyuk via Mike McCandless)
@@ -15352,13 +15354,13 @@ API Changes
     sub-sampling (over the termIndexInterval that was used during
     indexing) which terms are loaded into memory.  (Chuck Williams,
     Doug Cutting via Mike McCandless)
-    
+
  6. LUCENE-743: Add IndexReader.reopen() method that re-opens an
     existing IndexReader (see New features -> 8.) (Michael Busch)
 
- 7. LUCENE-1062: Add setData(byte[] data), 
+ 7. LUCENE-1062: Add setData(byte[] data),
     setData(byte[] data, int offset, int length), getData(), getOffset()
-    and clone() methods to o.a.l.index.Payload. Also add the field name 
+    and clone() methods to o.a.l.index.Payload. Also add the field name
     as arg to Similarity.scorePayload(). (Michael Busch)
 
  8. LUCENE-982: Add IndexWriter.optimize(int maxNumSegments) method to
@@ -15367,7 +15369,7 @@ API Changes
 
  9. LUCENE-1080: Changed Token.DEFAULT_TYPE to be public.
 
-10. LUCENE-1064: Changed TopDocs constructor to be public. 
+10. LUCENE-1064: Changed TopDocs constructor to be public.
      (Shai Erera via Michael Busch)
 
 11. LUCENE-1079: DocValues cleanup: constructor now has no params,
@@ -15376,10 +15378,10 @@ API Changes
 12. LUCENE-1089: Added PriorityQueue.insertWithOverflow, which returns
     the Object (if any) that was bumped from the queue to allow
     re-use.  (Shai Erera via Mike McCandless)
-    
+
 13. LUCENE-1101: Token reuse 'contract' (defined LUCENE-969)
     modified so it is token producer's responsibility
-    to call Token.clear(). (Doron Cohen)   
+    to call Token.clear(). (Doron Cohen)
 
 14. LUCENE-1118: Changed StandardAnalyzer to skip too-long (default >
     255 characters) tokens.  You can increase this limit by calling
@@ -15388,26 +15390,26 @@ API Changes
 
 Bug fixes
 
- 1. LUCENE-933: QueryParser fixed to not produce empty sub 
-    BooleanQueries "()" even if the Analyzer produced no 
+ 1. LUCENE-933: QueryParser fixed to not produce empty sub
+    BooleanQueries "()" even if the Analyzer produced no
     tokens for input. (Doron Cohen)
 
  2. LUCENE-955: Fixed SegmentTermPositions to work correctly with the
     first term in the dictionary. (Michael Busch)
 
  3. LUCENE-951: Fixed NullPointerException in MultiLevelSkipListReader
-    that was thrown after a call of TermPositions.seek(). 
+    that was thrown after a call of TermPositions.seek().
     (Rich Johnson via Michael Busch)
-    
+
  4. LUCENE-938: Fixed cases where an unhandled exception in
     IndexWriter's methods could cause deletes to be lost.
     (Steven Parkes via Mike McCandless)
-      
+
  5. LUCENE-962: Fixed case where an unhandled exception in
     IndexWriter.addDocument or IndexWriter.updateDocument could cause
     unreferenced files in the index to not be deleted
     (Steven Parkes via Mike McCandless)
-  
+
  6. LUCENE-957: RAMDirectory fixed to properly handle directories
     larger than Integer.MAX_VALUE. (Doron Cohen)
 
@@ -15415,7 +15417,7 @@ Bug fixes
     isOptimized() or getVersion() is called. Separated MultiReader
     into two classes: MultiSegmentReader extends IndexReader, is
     package-protected and is created automatically by IndexReader.open()
-    in case the index has multiple segments. The public MultiReader 
+    in case the index has multiple segments. The public MultiReader
     now extends MultiSegmentReader and is intended to be used by users
     who want to add their own subreaders. (Daniel Naber, Michael Busch)
 
@@ -15424,7 +15426,7 @@ Bug fixes
 
  9. LUCENE-832: ParallelReader fixed to not throw NPE if isCurrent(),
     isOptimized() or getVersion() is called. (Michael Busch)
-      
+
 10. LUCENE-948: Fix FNFE exception caused by stale NFS client
     directory listing caches when writers on different machines are
     sharing an index over NFS and using a custom deletion policy (Mike
@@ -15482,18 +15484,18 @@ Bug fixes
     Simple/NativeFSLockFactory if we fail to delete the lock file when
     releasing the lock.  (Nikolay Diakov via Mike McCandless)
 
-24. LUCENE-1071: Fixed SegmentMerger to correctly set payload bit in 
+24. LUCENE-1071: Fixed SegmentMerger to correctly set payload bit in
     the merged segment. (Michael Busch)
 
 25. LUCENE-1042: Remove throwing of IOException in getTermFreqVector(int, String, TermVectorMapper) to be consistent
     with other getTermFreqVector calls.  Also removed the throwing of the other IOException in that method to be consistent.  (Karl Wettin via Grant Ingersoll)
-    
-26. LUCENE-1096: Fixed Hits behavior when hits' docs are deleted 
-    along with iterating the hits. Deleting docs already retrieved 
-    now works seamlessly. If docs not yet retrieved are deleted 
-    (e.g. from another thread), and then, relying on the initial 
-    Hits.length(), an application attempts to retrieve more hits 
-    than actually exist , a ConcurrentMidificationException 
+
+26. LUCENE-1096: Fixed Hits behavior when hits' docs are deleted
+    along with iterating the hits. Deleting docs already retrieved
+    now works seamlessly. If docs not yet retrieved are deleted
+    (e.g. from another thread), and then, relying on the initial
+    Hits.length(), an application attempts to retrieve more hits
+    than actually exist , a ConcurrentMidificationException
     is thrown.  (Doron Cohen)
 
 27. LUCENE-1068: Changed StandardTokenizer to fix an issue with it marking
@@ -15502,15 +15504,15 @@ Bug fixes
   this flag to true fixes the problem.  This flag is a temporary fix and is already
   marked as being deprecated.  3.x will implement the correct approach.  (Shai Erera via Grant Ingersoll)
   LUCENE-1140: Fixed NPE caused by 1068 (Alexei Dets via Grant Ingersoll)
-    
-28. LUCENE-749: ChainedFilter behavior fixed when logic of 
+
+28. LUCENE-749: ChainedFilter behavior fixed when logic of
     first filter is ANDNOT.  (Antonio Bruno via Doron Cohen)
 
 29. LUCENE-508: Make sure SegmentTermEnum.prev() is accurate (= last
     term) after next() returns false.  (Steven Tamm via Mike
     McCandless)
 
-    
+
 New features
 
  1. LUCENE-906: Elision filter for French.
@@ -15558,7 +15560,7 @@ New features
     that have changed since the reader was (re)opened. reopen() can
     be significantly faster than open(), depending on the amount of
     index changes. SegmentReader, MultiSegmentReader, MultiReader,
-    and ParallelReader implement reopen(). (Michael Busch) 
+    and ParallelReader implement reopen(). (Michael Busch)
 
  9. LUCENE-1040: CharArraySet useful for efficiently checking
     set membership of text specified by char[]. (yonik)
@@ -15566,14 +15568,14 @@ New features
 10. LUCENE-1073: Created SnapshotDeletionPolicy to facilitate taking a
     live backup of an index without pausing indexing.  (Mike
     McCandless)
-    
-11. LUCENE-1019: CustomScoreQuery enhanced to support multiple 
+
+11. LUCENE-1019: CustomScoreQuery enhanced to support multiple
     ValueSource queries. (Kyle Maxwell via Doron Cohen)
-    
-12. LUCENE-1095: Added an option to StopFilter to increase 
+
+12. LUCENE-1095: Added an option to StopFilter to increase
     positionIncrement of the token succeeding a stopped token.
-    Disabled by default. Similar option added to QueryParser 
-    to consider token positions when creating PhraseQuery 
+    Disabled by default. Similar option added to QueryParser
+    to consider token positions when creating PhraseQuery
     and MultiPhraseQuery. Disabled by default (so by default
     the query parser ignores position increments).
     (Doron Cohen)
@@ -15584,9 +15586,9 @@ New features
 
 Optimizations
 
- 1. LUCENE-937: CachingTokenFilter now uses an iterator to access the 
-    Tokens that are cached in the LinkedList. This increases performance 
-    significantly, especially when the number of Tokens is large. 
+ 1. LUCENE-937: CachingTokenFilter now uses an iterator to access the
+    Tokens that are cached in the LinkedList. This increases performance
+    significantly, especially when the number of Tokens is large.
     (Mark Miller via Michael Busch)
 
  2. LUCENE-843: Substantial optimizations to improve how IndexWriter
@@ -15594,7 +15596,7 @@ Optimizations
     faster).  A single shared hash table now records the in-memory
     postings per unique term and is directly flushed into a single
     segment.  (Mike McCandless)
- 
+
  3. LUCENE-892: Fixed extra "buffer to buffer copy" that sometimes
     takes place when using compound files.  (Mike McCandless)
 
@@ -15621,8 +15623,8 @@ Optimizations
 
 10. LUCENE-986: Refactored SegmentInfos from IndexReader into the new
     subclass DirectoryIndexReader. SegmentReader and MultiSegmentReader
-    now extend DirectoryIndexReader and are the only IndexReader 
-    implementations that use SegmentInfos to access an index and 
+    now extend DirectoryIndexReader and are the only IndexReader
+    implementations that use SegmentInfos to access an index and
     acquire a write lock for index modifications. (Michael Busch)
 
 11. LUCENE-1007: Allow flushing in IndexWriter to be triggered by
@@ -15638,13 +15640,13 @@ Optimizations
     documents, and a slight performance increase for top level
     conjunctions.  (yonik)
 
-14. LUCENE-1098: Make inner class StandardAnalyzer.SavedStreams static 
+14. LUCENE-1098: Make inner class StandardAnalyzer.SavedStreams static
     and final. (Nathan Beyer via Michael Busch)
 
 Documentation
 
  1. LUCENE-1051: Generate separate javadocs for core, demo and contrib
-    classes, as well as an unified view. Also add an appropriate menu 
+    classes, as well as an unified view. Also add an appropriate menu
     structure to the website. (Michael Busch)
 
  2. LUCENE-746: Fix error message in AnalyzingQueryParser.getPrefixQuery.
@@ -15664,26 +15666,26 @@ Build
     line.  This should be helpful for easy application of patches, but it
     is also a step towards integrating automatic patch application with
     JIRA and Hudson, and is thus subject to change.  (Grant Ingersoll)
- 
+
  4. LUCENE-935: Defined property "m2.repository.url" to allow setting
     the url to a maven remote repository to deploy to. (Michael Busch)
 
  5. LUCENE-1051: Include javadocs in the maven artifacts. (Michael Busch)
 
- 6. LUCENE-1055: Remove gdata-server from build files and its sources 
+ 6. LUCENE-1055: Remove gdata-server from build files and its sources
     from trunk. (Michael Busch)
 
  7. LUCENE-935: Allow to deploy maven artifacts to a remote m2 repository
     via scp and ssh authentication. (Michael Busch)
-  
- 8. LUCENE-1123: Allow overriding the specification version for 
+
+ 8. LUCENE-1123: Allow overriding the specification version for
     MANIFEST.MF (Michael Busch)
 
 Test Cases
 
- 1. LUCENE-766: Test adding two fields with the same name but different 
-    term vector setting.  (Nicolas Lalevée via Doron Cohen)  
-    
+ 1. LUCENE-766: Test adding two fields with the same name but different
+    term vector setting.  (Nicolas Lalevée via Doron Cohen)
+
 ======================= Release 2.2.0 =======================
 
 Changes in runtime behavior
@@ -15717,7 +15719,7 @@ API Changes
 
  5. LUCENE-834: Changed some access levels for certain Span classes to allow them
     to be overridden.  They have been marked expert only and not for public
-    consumption. (Grant Ingersoll) 
+    consumption. (Grant Ingersoll)
 
  6. LUCENE-796: Removed calls to super.* from various get*Query methods in
     MultiFieldQueryParser, in order to allow sub-classes to override them.
@@ -15736,13 +15738,13 @@ API Changes
     the TokenStream more than once. (Michael Busch)
 
 10. LUCENE-580: Added a new constructor to Field that takes a TokenStream as
-    argument, available as tokenStreamValue(). This is useful to avoid the need of 
+    argument, available as tokenStreamValue(). This is useful to avoid the need of
     "dummy analyzers" for pre-analyzed fields. (Karl Wettin, Michael Busch)
 
 11. LUCENE-730: Added the new methods to BooleanQuery setAllowDocsOutOfOrder() and
-    getAllowDocsOutOfOrder(). Deprecated the methods setUseScorer14() and 
-    getUseScorer14(). The optimization patch LUCENE-730 (see Optimizations->3.) 
-    improves performance for certain queries but results in scoring out of docid 
+    getAllowDocsOutOfOrder(). Deprecated the methods setUseScorer14() and
+    getUseScorer14(). The optimization patch LUCENE-730 (see Optimizations->3.)
+    improves performance for certain queries but results in scoring out of docid
     order. This patch reverse this change, so now by default hit docs are scored
     in docid order if not setAllowDocsOutOfOrder(true) is explicitly called.
     This patch also enables the tests in QueryUtils again that check for docid
@@ -15762,8 +15764,8 @@ Bug fixes
  1. LUCENE-804: Fixed build.xml to pack a fully compilable src dist.  (Doron Cohen)
 
  2. LUCENE-813: Leading wildcard fixed to work with trailing wildcard.
-    Query parser modified to create a prefix query only for the case 
-    that there is a single trailing wildcard (and no additional wildcard 
+    Query parser modified to create a prefix query only for the case
+    that there is a single trailing wildcard (and no additional wildcard
     or '?' in the query text).  (Doron Cohen)
 
  3. LUCENE-812: Add no-argument constructors to NativeFSLockFactory
@@ -15785,19 +15787,19 @@ Bug fixes
     a FileNotFoundException like Lucene pre-2.1 (before this fix you
     got an NPE).  (Mike McCandless)
 
- 7. LUCENE-800: Removed backslash from the TERM_CHAR list in the queryparser, 
+ 7. LUCENE-800: Removed backslash from the TERM_CHAR list in the queryparser,
     because the backslash is the escape character. Also changed the ESCAPED_CHAR
-    list to contain all possible characters, because every character that 
+    list to contain all possible characters, because every character that
     follows a backslash should be considered as escaped. (Michael Busch)
 
- 8. LUCENE-372: QueryParser.parse() now ensures that the entire input string 
+ 8. LUCENE-372: QueryParser.parse() now ensures that the entire input string
     is consumed. Now a ParseException is thrown if a query contains too many
     closing parentheses. (Andreas Neumann via Michael Busch)
 
  9. LUCENE-814: javacc build targets now fix line-end-style of generated files.
     Now also deleting all javacc generated files before calling javacc.
     (Steven Parkes, Doron Cohen)
-    
+
 10. LUCENE-829: close readers in contrib/benchmark. (Karl Wettin, Doron Cohen)
 
 11. LUCENE-828: Minor fix for Term's equal().
@@ -15809,18 +15811,18 @@ Bug fixes
     the instance of IndexWriter (but, not the index itself) by
     referencing already deleted segments.  This bug was only present
     in 2.2 (trunk), ie was never released.  (Mike McCandless)
-    
+
 13. LUCENE-736: Sloppy phrase query with repeating terms matches wrong docs.
     For example query "B C B"~2 matches the doc "A B C D E". (Doron Cohen)
-    
-14. LUCENE-789: Fixed: custom similarity is ignored when using MultiSearcher (problem reported 
-    by Alexey Lef). Now the similarity applied by MultiSearcer.setSimilarity(sim) is being used. 
-    Note that as before this fix, creating a multiSearcher from Searchers for whom custom similarity 
-    was set has no effect - it is masked by the similarity of the MultiSearcher. This is as 
+
+14. LUCENE-789: Fixed: custom similarity is ignored when using MultiSearcher (problem reported
+    by Alexey Lef). Now the similarity applied by MultiSearcer.setSimilarity(sim) is being used.
+    Note that as before this fix, creating a multiSearcher from Searchers for whom custom similarity
+    was set has no effect - it is masked by the similarity of the MultiSearcher. This is as
     designed, because MultiSearcher operates on Searchables (not Searchers). (Doron Cohen)
 
 15. LUCENE-880: Fixed DocumentWriter to close the TokenStreams after it
-    has written the postings. Then the resources associated with the 
+    has written the postings. Then the resources associated with the
     TokenStreams can safely be released. (Michael Busch)
 
 16. LUCENE-883: consecutive calls to Spellchecker.indexDictionary()
@@ -15833,21 +15835,21 @@ Bug fixes
     anymore and ignored, but re-thrown. Some javadoc improvements.
     (Daniel Naber)
 
-19. LUCENE-698: FilteredQuery now takes the query boost into account for 
+19. LUCENE-698: FilteredQuery now takes the query boost into account for
     scoring. (Michael Busch)
 
-20. LUCENE-763: Spellchecker: LuceneDictionary used to skip first word in 
+20. LUCENE-763: Spellchecker: LuceneDictionary used to skip first word in
     enumeration. (Christian Mallwitz via Daniel Naber)
-    
+
 21. LUCENE-903: FilteredQuery explanation inaccuracy with boost.
     Explanation tests now "deep" check the explanation details.
     (Chris Hostetter, Doron Cohen)
-    
-22. LUCENE-912: DisjunctionMaxScorer first skipTo(target) call ignores the 
+
+22. LUCENE-912: DisjunctionMaxScorer first skipTo(target) call ignores the
     skip target param and ends up at the first match.
     (Sudaakeran B. via Chris Hostetter & Doron Cohen)
-    
-23. LUCENE-913: Two consecutive score() calls return different 
+
+23. LUCENE-913: Two consecutive score() calls return different
     scores for Boolean Queries. (Michael Busch, Doron Cohen)
 
 24. LUCENE-1013: Fix IndexWriter.setMaxMergeDocs to work "out of the
@@ -15885,7 +15887,7 @@ New features
     on the remote side of the RMI connection.
     (Matt Ericson via Otis Gospodnetic)
 
- 8. LUCENE-446: Added Solr's search.function for scores based on field 
+ 8. LUCENE-446: Added Solr's search.function for scores based on field
     values, plus CustomScoreQuery for simple score (post) customization.
     (Yonik Seeley, Doron Cohen)
 
@@ -15894,7 +15896,7 @@ New features
     Fields that share all the same analysis steps except one lowercases tokens and the other does not, you can coordinate the operations
     between the two using the TeeTokenFilter and the SinkTokenizer.  See TeeSinkTokenTest.java for examples.
     (Grant Ingersoll, Michael Busch, Yonik Seeley)
- 
+
 Optimizations
 
  1. LUCENE-761: The proxStream is now cloned lazily in SegmentTermPositions
@@ -15903,7 +15905,7 @@ Optimizations
     (Michael Busch)
 
  2. LUCENE-431: RAMInputStream and RAMOutputStream extend IndexInput and
-    IndexOutput directly now. This avoids further buffering and thus avoids 
+    IndexOutput directly now. This avoids further buffering and thus avoids
     unnecessary array copies. (Michael Busch)
 
  3. LUCENE-730: Updated BooleanScorer2 to make use of BooleanScorer in some
@@ -15919,16 +15921,16 @@ Optimizations
  5. LUCENE-430: Delay allocation of the buffer after a clone of BufferedIndexInput.
     Together with LUCENE-888 this will allow to adjust the buffer size
     dynamically. (Paul Elschot, Michael Busch)
- 
+
  6. LUCENE-888: Increase buffer sizes inside CompoundFileWriter and
     BufferedIndexOutput.  Also increase buffer size in
     BufferedIndexInput, but only when used during merging.  Together,
     these increases yield 10-18% overall performance gain vs the
     previous 1K defaults.  (Mike McCandless)
 
- 7. LUCENE-866: Adds multi-level skip lists to the posting lists. This speeds 
-    up most queries that use skipTo(), especially on big indexes with large posting 
-    lists. For average AND queries the speedup is about 20%, for queries that 
+ 7. LUCENE-866: Adds multi-level skip lists to the posting lists. This speeds
+    up most queries that use skipTo(), especially on big indexes with large posting
+    lists. For average AND queries the speedup is about 20%, for queries that
     contain very frequent and very unique terms the speedup can be over 80%.
     (Michael Busch)
 
@@ -15938,12 +15940,12 @@ Documentation
     http://wiki.apache.org/lucene-java/   Updated the links in the docs and
     wherever else I found references.  (Grant Ingersoll, Joe Schaefer)
 
- 2. LUCENE-807: Fixed the javadoc for ScoreDocComparator.compare() to be 
-    consistent with java.util.Comparator.compare(): Any integer is allowed to 
+ 2. LUCENE-807: Fixed the javadoc for ScoreDocComparator.compare() to be
+    consistent with java.util.Comparator.compare(): Any integer is allowed to
     be returned instead of only -1/0/1.
     (Paul Cowan via Michael Busch)
- 
- 3. LUCENE-875: Solved javadoc warnings & errors under jdk1.4. 
+
+ 3. LUCENE-875: Solved javadoc warnings & errors under jdk1.4.
     Solved javadoc errors under jdk5 (jars in path for gdata).
     Made "javadocs" target depend on "build-contrib" for first downloading
     contrib jars configured for dynamic downloaded. (Note: when running
@@ -15963,9 +15965,9 @@ Build
 
  2. LUCENE-885: "ant test" now includes all contrib tests.  The new
     "ant test-core" target can be used to run only the Core (non
-    contrib) tests. 
+    contrib) tests.
     (Chris Hostetter)
-    
+
  3. LUCENE-900: "ant test" now enables Java assertions (in Lucene packages).
     (Doron Cohen)
 
@@ -15977,7 +15979,7 @@ Build
 
  6. LUCENE-907: Include LICENSE.TXT and NOTICE.TXT in the META-INF dirs of
     demo war, demo jar, and the contrib jars. (Michael Busch)
-    
+
  7. LUCENE-909: Demo targets for running the demo. (Doron Cohen)
 
  8. LUCENE-908: Improves content of MANIFEST file and makes it customizable
@@ -15989,8 +15991,8 @@ Build
     dependencies are met, and test compilation errors fail the build.
     (Steven Parkes, Chris Hostetter)
 
-10. LUCENE-622: Add ant target and pom.xml files for building maven artifacts 
-    of the Lucene core and the contrib modules. 
+10. LUCENE-622: Add ant target and pom.xml files for building maven artifacts
+    of the Lucene core and the contrib modules.
     (Sami Siren, Karl Wettin, Michael Busch)
 
 ======================= Release 2.1.0 =======================
@@ -16332,7 +16334,7 @@ Bug fixes
 
 31. Remove "tvp" from known index file extensions because it is
     never used. (Nicolas Lalevée via Bernhard Messer)
-    
+
 32. LUCENE-767: Change how SegmentReader.maxDoc() is computed to not
     rely on file length check and instead use the SegmentInfo's
     docCount that's already stored explicitly in the index.  This is a
@@ -16424,7 +16426,7 @@ Test Cases
 
   2. Added TestWindowsMMap.java (Benson Margulies via Mike McCandless)
 
-  3. LUCENE-744 Append the user.name property onto the temporary directory 
+  3. LUCENE-744 Append the user.name property onto the temporary directory
      that is created so it doesn't interfere with other users. (Grant Ingersoll)
 
 Documentation
@@ -16478,7 +16480,7 @@ Build
      ${build.dir}/test just like the tempDir sysproperty.
 
   3. LUCENE-757 Added new target named init-dist that does setup for
-     distribution of both binary and source distributions.  Called by package 
+     distribution of both binary and source distributions.  Called by package
      and package-*-src
 
 ======================= Release 2.0.0 =======================
@@ -16510,7 +16512,7 @@ New features
 
  2. LUCENE-577: SweetSpotSimilarity added to contrib/miscellaneous.
     (Chris Hostetter)
-    
+
 Bug fixes
 
  1. LUCENE-330: Fix issue of FilteredQuery not working properly within
@@ -16587,14 +16589,14 @@ been deprecated in anticipation of release 2.0, so deprecation
 warnings are to be expected when upgrading from 1.4.3 to 1.9.
 
 Bug fixes
- 
- 1. The fix that made IndexWriter.setMaxBufferedDocs(1) work had negative 
-    effects on indexing performance and has thus been reverted. The 
+
+ 1. The fix that made IndexWriter.setMaxBufferedDocs(1) work had negative
+    effects on indexing performance and has thus been reverted. The
     argument for setMaxBufferedDocs(int) must now at least be 2, otherwise
     an exception is thrown. (Daniel Naber)
- 
+
 Optimizations
-     
+
  1. Optimized BufferedIndexOutput.writeBytes() to use
     System.arraycopy() in more cases, rather than copying byte-by-byte.
     (Lukas Zapletal via Cutting)
@@ -16925,7 +16927,7 @@ Bug fixes
 22. IndexWriter.setMaxBufferedDocs(1) didn't have the expected effect,
     this has now been fixed. (Daniel Naber)
 
-23. Fixed QueryParser when called with a date in local form like 
+23. Fixed QueryParser when called with a date in local form like
     "[1/16/2000 TO 1/18/2000]". This query did not include the documents
     of 1/18/2000, i.e. the last day was not included. (Daniel Naber)
 
@@ -16933,9 +16935,9 @@ Bug fixes
     not yet any values for the sort field (Yonik Seeley, LUCENE-374)
 
 Optimizations
-     
+
  1. Disk usage (peak requirements during indexing and optimization)
-    in case of compound file format has been improved. 
+    in case of compound file format has been improved.
     (Bernhard, Dmitry, and Christoph)
 
  2. Optimize the performance of certain uses of BooleanScorer,
@@ -16943,7 +16945,7 @@ Optimizations
     composed of TermQuery, with not all terms required, that returns a
     TopDocs (e.g., through a Hits with no Sort specified) runs much
     faster.  (cutting)
-    
+
  3. Removed synchronization from reading of term vectors with an
     IndexReader (Patch #30736). (Bernhard Messer via Christoph)
 
@@ -16952,10 +16954,10 @@ Optimizations
     low-frequency terms, where the cost of dictionary lookup can be
     significant. (cutting)
 
- 5. Optimize fuzzy queries so the standard fuzzy queries with a prefix 
+ 5. Optimize fuzzy queries so the standard fuzzy queries with a prefix
     of 0 now run 20-50% faster (Patch #31882).
     (Jonathan Hager via Daniel Naber)
-    
+
  6. A Version of BooleanScorer (BooleanScorer2) added that delivers
     documents in increasing order and implements skipTo. For queries
     with required or forbidden clauses it may be faster than the old
@@ -17004,13 +17006,13 @@ Infrastructure
 1.4.3
 
  1. The JSP demo page (src/jsp/results.jsp) now properly escapes error
-    messages which might contain user input (e.g. error messages about 
+    messages which might contain user input (e.g. error messages about
     query parsing). If you used that page as a starting point for your
     own code please make sure your code also properly escapes HTML
     characters from user input in order to avoid so-called cross site
     scripting attacks. (Daniel Naber)
-  
-  2. QueryParser changes in 1.4.2 broke the QueryParser API. Now the old 
+
+  2. QueryParser changes in 1.4.2 broke the QueryParser API. Now the old
      API is supported again. (Christoph)
 
 
@@ -17022,20 +17024,20 @@ Infrastructure
 
  2. Memory leak in Sort code (bug #31240) eliminated.
     (Rafal Krzewski via Christoph and Daniel)
-    
+
  3. FuzzyQuery now takes an additional parameter that specifies the
     minimum similarity that is required for a term to match the query.
-    The QueryParser syntax for this is term~x, where x is a floating 
+    The QueryParser syntax for this is term~x, where x is a floating
     point number >= 0 and < 1 (a bigger number means that a higher
     similarity is required). Furthermore, a prefix can be specified
-    for FuzzyQuerys so that only those terms are considered similar that 
+    for FuzzyQuerys so that only those terms are considered similar that
     start with this prefix. This can speed up FuzzyQuery greatly.
     (Daniel Naber, Christoph Goller)
-    
+
  4. PhraseQuery and PhrasePrefixQuery now allow the explicit specification
     of relative positions. (Christoph Goller)
-    
- 5. QueryParser changes: Fix for ArrayIndexOutOfBoundsExceptions 
+
+ 5. QueryParser changes: Fix for ArrayIndexOutOfBoundsExceptions
     (patch #9110); some unused method parameters removed; The ability
     to specify a minimum similarity for FuzzyQuery has been added.
     (Christoph Goller)
diff --git a/lucene/core/src/java/org/apache/lucene/document/ShapeField.java b/lucene/core/src/java/org/apache/lucene/document/ShapeField.java
index f3e89d63275..da34aa2eeaa 100644
--- a/lucene/core/src/java/org/apache/lucene/document/ShapeField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/ShapeField.java
@@ -332,6 +332,8 @@ public final class ShapeField {
       if (triangle.aX == triangle.cX && triangle.aY == triangle.cY) {
         triangle.type = DecodedTriangle.TYPE.POINT;
       } else {
+        // a and b are identical, remove ab, and merge bc and ca
+        triangle.ab = triangle.bc | triangle.ca;
         triangle.bX = triangle.cX;
         triangle.bY = triangle.cY;
         triangle.cX = triangle.aX;
@@ -339,8 +341,12 @@ public final class ShapeField {
         triangle.type = DecodedTriangle.TYPE.LINE;
       }
     } else if (triangle.aX == triangle.cX && triangle.aY == triangle.cY) {
+      // a and c are identical, remove ac, and merge ab and bc
+      triangle.ab = triangle.ab | triangle.bc;
       triangle.type = DecodedTriangle.TYPE.LINE;
     } else if (triangle.bX == triangle.cX && triangle.bY == triangle.cY) {
+      // b and c are identical, remove bc, and merge ab and ca
+      triangle.ab = triangle.ab | triangle.ca;
       triangle.cX = triangle.aX;
       triangle.cY = triangle.aY;
       triangle.type = DecodedTriangle.TYPE.LINE;
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestLatLonShape.java b/lucene/core/src/test/org/apache/lucene/document/TestLatLonShape.java
index 44c293f55cb..c69c0edc82e 100644
--- a/lucene/core/src/test/org/apache/lucene/document/TestLatLonShape.java
+++ b/lucene/core/src/test/org/apache/lucene/document/TestLatLonShape.java
@@ -887,8 +887,8 @@ public class TestLatLonShape extends LuceneTestCase {
     IndexReader reader = w.getReader();
     w.close();
     IndexSearcher searcher = newSearcher(reader);
-    // Contains is only true if the query geometry is inside a geometry and does not intersect with any other geometry 
-    // belonging to the same document. In this case the query geometry contains the indexed polygon but the point is 
+    // Contains is only true if the query geometry is inside a geometry and does not intersect with any other geometry
+    // belonging to the same document. In this case the query geometry contains the indexed polygon but the point is
     // inside the query as well, hence the result is 0.
     Polygon polygonQuery = new Polygon(new double[] {4, 4, 6, 6, 4}, new double[] {4, 6, 6, 4, 4});
     Query query = LatLonShape.newGeometryQuery(FIELDNAME, QueryRelation.CONTAINS, polygonQuery);
@@ -897,11 +897,39 @@ public class TestLatLonShape extends LuceneTestCase {
     Rectangle rectangle = new Rectangle(4.0, 6.0, 4.0, 6.0);
     query = LatLonShape.newGeometryQuery(FIELDNAME, QueryRelation.CONTAINS, rectangle);
     assertEquals(0, searcher.count(query));
-    
+
     Circle circle = new Circle(5, 5, 10000);
     query = LatLonShape.newGeometryQuery(FIELDNAME, QueryRelation.CONTAINS, circle);
     assertEquals(0, searcher.count(query));
-    
+
     IOUtils.close(w, reader, dir);
   }
+
+  public void testFlatPolygonDoesNotContainIntersectingLine() throws Exception {
+    // Create line intersecting very flat polygon (but not contained)
+    double[] lons = new double[] {-0.001, -0.001, 0.001, 0.001, -0.001};
+    double[] lats = new double[] {1e-10, 0, -1e-10, 0, 1e-10};
+    Polygon polygon = new Polygon(lats, lons);
+    Line line = new Line(new double[] {0.0, 0.001}, new double[] {0.0, 0.0});
+
+    // Index the polygon
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    Document document = new Document();
+    addPolygonsToDoc(FIELDNAME, document, polygon);
+    writer.addDocument(document);
+
+    // search
+    IndexReader r = writer.getReader();
+    writer.close();
+    IndexSearcher s = newSearcher(r);
+
+    // search for line within the polygon
+    Query q = LatLonShape.newGeometryQuery(FIELDNAME, QueryRelation.CONTAINS, line);
+    TopDocs topDocs = s.search(q, 1);
+    assertEquals("Polygon should not contain the line,", 0, topDocs.scoreDocs.length);
+
+    // cleanup
+    IOUtils.close(r, dir);
+  }
 }