You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2012/04/22 23:15:28 UTC

svn commit: r1328978 - in /lucene/dev/trunk/lucene: JRE_VERSION_MIGRATION.txt MIGRATE.txt README.txt build.xml common-build.xml site/xsl/index.xsl

Author: uschindler
Date: Sun Apr 22 21:15:27 2012
New Revision: 1328978

URL: http://svn.apache.org/viewvc?rev=1328978&view=rev
Log:
LUCENE-4008: Use pegdown to transform MIGRATE.txt and other text-only files to readable HTML. Please alsows run ant documentation when you have changed anything on those files to check output.

Modified:
    lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt
    lucene/dev/trunk/lucene/MIGRATE.txt
    lucene/dev/trunk/lucene/README.txt
    lucene/dev/trunk/lucene/build.xml
    lucene/dev/trunk/lucene/common-build.xml
    lucene/dev/trunk/lucene/site/xsl/index.xsl

Modified: lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt?rev=1328978&r1=1328977&r2=1328978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt (original)
+++ lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt Sun Apr 22 21:15:27 2012
@@ -1,36 +1,37 @@
+# JRE Version Migration Guide
+
 If possible, use the same JRE major version at both index and search time.
 When upgrading to a different JRE major version, consider re-indexing. 
 
 Different JRE major versions may implement different versions of Unicode,
 which will change the way some parts of Lucene treat your text.
 
-For example: with Java 1.4, LetterTokenizer will split around the character U+02C6,
+For example: with Java 1.4, `LetterTokenizer` will split around the character U+02C6,
 but with Java 5 it will not.
 This is because Java 1.4 implements Unicode 3, but Java 5 implements Unicode 4.
 
 For reference, JRE major versions with their corresponding Unicode versions:
-Java 1.4, Unicode 3.0
-Java 5, Unicode 4.0
-Java 6, Unicode 4.0
-Java 7, Unicode 6.0
+
+ * Java 1.4, Unicode 3.0
+ * Java 5, Unicode 4.0
+ * Java 6, Unicode 4.0
+ * Java 7, Unicode 6.0
 
 In general, whether or not you need to re-index largely depends upon the data that
 you are searching, and what was changed in any given Unicode version. For example, 
 if you are completely sure that your content is limited to the "Basic Latin" range 
 of Unicode, you can safely ignore this. 
 
-Special Notes:
-
-LUCENE 2.9 TO 3.0, JAVA 1.4 TO JAVA 5 TRANSITION
+## Special Notes: LUCENE 2.9 TO 3.0, JAVA 1.4 TO JAVA 5 TRANSITION
 
-* StandardAnalyzer will return the same results under Java 5 as it did under 
+* `StandardAnalyzer` will return the same results under Java 5 as it did under 
 Java 1.4. This is because it is largely independent of the runtime JRE for
 Unicode support, (with the exception of lowercasing).  However, no changes to 
 casing have occurred in Unicode 4.0 that affect StandardAnalyzer, so if you are 
 using this Analyzer you are NOT affected.
 
-* SimpleAnalyzer, StopAnalyzer, LetterTokenizer, LowerCaseFilter, and 
-LowerCaseTokenizer may return different results, along with many other Analyzers
-and TokenStreams in Lucene's analysis modules. If you are using one of these 
+* `SimpleAnalyzer`, `StopAnalyzer`, `LetterTokenizer`, `LowerCaseFilter`, and 
+`LowerCaseTokenizer` may return different results, along with many other `Analyzer`s
+and `TokenStream`s in Lucene's analysis modules. If you are using one of these 
 components, you may be affected.
 

Modified: lucene/dev/trunk/lucene/MIGRATE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/MIGRATE.txt?rev=1328978&r1=1328977&r2=1328978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/MIGRATE.txt (original)
+++ lucene/dev/trunk/lucene/MIGRATE.txt Sun Apr 22 21:15:27 2012
@@ -1,138 +1,63 @@
+# Apache Lucene Migration Guide
 
-LUCENE-2380: FieldCache.getStrings/Index --> FieldCache.getDocTerms/Index
+## Four-dimensional enumerations
 
-  * The field values returned when sorting by SortField.STRING are now
-    BytesRef.  You can call value.utf8ToString() to convert back to
-    string, if necessary.
-
-  * In FieldCache, getStrings (returning String[]) has been replaced
-    with getTerms (returning a FieldCache.DocTerms instance).
-    DocTerms provides a getTerm method, taking a docID and a BytesRef
-    to fill (which must not be null), and it fills it in with the
-    reference to the bytes for that term.
-
-    If you had code like this before:
-
-      String[] values = FieldCache.DEFAULT.getStrings(reader, field);
-      ...
-      String aValue = values[docID];
-
-    you can do this instead:
-
-      DocTerms values = FieldCache.DEFAULT.getTerms(reader, field);
-      ...
-      BytesRef term = new BytesRef();
-      String aValue = values.getTerm(docID, term).utf8ToString();
-
-    Note however that it can be costly to convert to String, so it's
-    better to work directly with the BytesRef.
-
-  * Similarly, in FieldCache, getStringIndex (returning a StringIndex
-    instance, with direct arrays int[] order and String[] lookup) has
-    been replaced with getTermsIndex (returning a
-    FieldCache.DocTermsIndex instance).  DocTermsIndex provides the
-    getOrd(int docID) method to lookup the int order for a document,
-    lookup(int ord, BytesRef reuse) to lookup the term from a given
-    order, and the sugar method getTerm(int docID, BytesRef reuse)
-    which internally calls getOrd and then lookup.
-
-    If you had code like this before:
-
-      StringIndex idx = FieldCache.DEFAULT.getStringIndex(reader, field);
-      ...
-      int ord = idx.order[docID];
-      String aValue = idx.lookup[ord];
-
-    you can do this instead:
-
-      DocTermsIndex idx = FieldCache.DEFAULT.getTermsIndex(reader, field);
-      ...
-      int ord = idx.getOrd(docID);
-      BytesRef term = new BytesRef();
-      String aValue = idx.lookup(ord, term).utf8ToString();
-
-    Note however that it can be costly to convert to String, so it's
-    better to work directly with the BytesRef.
-
-    DocTermsIndex also has a getTermsEnum() method, which returns an
-    iterator (TermsEnum) over the term values in the index (ie,
-    iterates ord = 0..numOrd()-1).
-
-  * StringComparatorLocale is now more CPU costly than it was before
-    (it was already very CPU costly since it does not compare using
-    indexed collation keys; use CollationKeyFilter for better
-    performance), since it converts BytesRef -> String on the fly.
-    Also, the field values returned when sorting by SortField.STRING
-    are now BytesRef.
-
-  * FieldComparator.StringOrdValComparator has been renamed to
-    TermOrdValComparator, and now uses BytesRef for its values.
-    Likewise for StringValComparator, renamed to TermValComparator.
-    This means when sorting by SortField.STRING or
-    SortField.STRING_VAL (or directly invoking these comparators) the
-    values returned in the FieldDoc.fields array will be BytesRef not
-    String.  You can call the .utf8ToString() method on the BytesRef
-    instances, if necessary.
+Flexible indexing changed the low level fields/terms/docs/positions
+enumeration APIs.  Here are the major changes:
 
+  * Terms are now binary in nature (arbitrary byte[]), represented
+    by the BytesRef class (which provides an offset + length "slice"
+    into an existing byte[]).
 
-
-LUCENE-1458, LUCENE-2111: Flexible Indexing
-
-  Flexible indexing changed the low level fields/terms/docs/positions
-  enumeration APIs.  Here are the major changes:
-
-    * Terms are now binary in nature (arbitrary byte[]), represented
-      by the BytesRef class (which provides an offset + length "slice"
-      into an existing byte[]).
-
-    * Fields are separately enumerated (FieldsEnum) from the terms
-      within each field (TermEnum).  So instead of this:
+  * Fields are separately enumerated (FieldsEnum) from the terms
+    within each field (TermEnum).  So instead of this:
 
         TermEnum termsEnum = ...;
-	while(termsEnum.next()) {
-	  Term t = termsEnum.term();
-	  System.out.println("field=" + t.field() + "; text=" + t.text());
+        while(termsEnum.next()) {
+          Term t = termsEnum.term();
+          System.out.println("field=" + t.field() + "; text=" + t.text());
         }
 
-      Do this:
- 
+    Do this:
+
         FieldsEnum fieldsEnum = ...;
-	String field;
-	while((field = fieldsEnum.next()) != null) {
-	  TermsEnum termsEnum = fieldsEnum.terms();
-	  BytesRef text;
-	  while((text = termsEnum.next()) != null) {
-	    System.out.println("field=" + field + "; text=" + text.utf8ToString());
-	  }
+        String field;
+        while((field = fieldsEnum.next()) != null) {
+            TermsEnum termsEnum = fieldsEnum.terms();
+            BytesRef text;
+            while((text = termsEnum.next()) != null) {
+              System.out.println("field=" + field + "; text=" + text.utf8ToString());
+          }
+        }
 
-    * TermDocs is renamed to DocsEnum.  Instead of this:
+  * TermDocs is renamed to DocsEnum.  Instead of this:
 
         while(td.next()) {
-	  int doc = td.doc();
-	  ...
-	}
+          int doc = td.doc();
+          ...
+        }
 
-      do this:
+    do this:
 
         int doc;
-	while((doc = td.next()) != DocsEnum.NO_MORE_DOCS) {
-	  ...
- 	}
+        while((doc = td.next()) != DocsEnum.NO_MORE_DOCS) {
+          ...
+        }
 
-      Instead of this:
-      
+    Instead of this:
+    
         if (td.skipTo(target)) {
-	  int doc = td.doc();
-	  ...
-	}
+          int doc = td.doc();
+          ...
+        }
 
-      do this:
-      
+    do this:
+    
         if ((doc=td.advance(target)) != DocsEnum.NO_MORE_DOCS) {
-	  ...
-	}
+          ...
+        }
 
-      The bulk read API has also changed.  Instead of this:
+    The bulk read API has also changed.  Instead of this:
 
         int[] docs = new int[256];
         int[] freqs = new int[256];
@@ -145,7 +70,7 @@ LUCENE-1458, LUCENE-2111: Flexible Index
           // use docs[i], freqs[i]
         }
 
-      do this:
+    do this:
 
         DocsEnum.BulkReadResult bulk = td.getBulkResult();
         while(true) {
@@ -156,319 +81,358 @@ LUCENE-1458, LUCENE-2111: Flexible Index
           // use bulk.docs.ints[i] and bulk.freqs.ints[i]
         }
 
-    * TermPositions is renamed to DocsAndPositionsEnum, and no longer
-      extends the docs only enumerator (DocsEnum).
+  * TermPositions is renamed to DocsAndPositionsEnum, and no longer
+    extends the docs only enumerator (DocsEnum).
 
-    * Deleted docs are no longer implicitly filtered from
-      docs/positions enums.  Instead, you pass a Bits
-      skipDocs (set bits are skipped) when obtaining the enums.  Also,
-      you can now ask a reader for its deleted docs.
-
-    * The docs/positions enums cannot seek to a term.  Instead,
-      TermsEnum is able to seek, and then you request the
-      docs/positions enum from that TermsEnum.
+  * Deleted docs are no longer implicitly filtered from
+    docs/positions enums.  Instead, you pass a Bits
+    skipDocs (set bits are skipped) when obtaining the enums.  Also,
+    you can now ask a reader for its deleted docs.
+
+  * The docs/positions enums cannot seek to a term.  Instead,
+    TermsEnum is able to seek, and then you request the
+    docs/positions enum from that TermsEnum.
 
-    * TermsEnum's seek method returns more information.  So instead of
-      this:
+  * TermsEnum's seek method returns more information.  So instead of
+    this:
 
         Term t;
         TermEnum termEnum = reader.terms(t);
-	if (t.equals(termEnum.term())) {
-	  ...
+        if (t.equals(termEnum.term())) {
+          ...
         }
 
-      do this:
+    do this:
 
         TermsEnum termsEnum = ...;
-	BytesRef text;
-	if (termsEnum.seek(text) == TermsEnum.SeekStatus.FOUND) {
-	  ...
-	}
-
-      SeekStatus also contains END (enumerator is done) and NOT_FOUND
-      (term was not found but enumerator is now positioned to the next
-      term).
-
-    * TermsEnum has an ord() method, returning the long numeric
-      ordinal (ie, first term is 0, next is 1, and so on) for the term
-      it's not positioned to.  There is also a corresponding seek(long
-      ord) method.  Note that these methods are optional; in
-      particular the MultiFields TermsEnum does not implement them.
-
-
-  How you obtain the enums has changed.  The primary entry point is
-  the Fields class.  If you know your reader is a single segment
-  reader, do this:
-
-    Fields fields = reader.Fields();
-    if (fields != null) {
-      ...
-    }
+        BytesRef text;
+        if (termsEnum.seek(text) == TermsEnum.SeekStatus.FOUND) {
+          ...
+        }
 
-  If the reader might be multi-segment, you must do this:
+    SeekStatus also contains END (enumerator is done) and NOT_FOUND
+    (term was not found but enumerator is now positioned to the next
+    term).
+
+  * TermsEnum has an ord() method, returning the long numeric
+    ordinal (ie, first term is 0, next is 1, and so on) for the term
+    it's not positioned to.  There is also a corresponding seek(long
+    ord) method.  Note that these methods are optional; in
+    particular the MultiFields TermsEnum does not implement them.
+
+
+  * How you obtain the enums has changed.  The primary entry point is
+    the Fields class.  If you know your reader is a single segment
+    reader, do this:
+
+        Fields fields = reader.Fields();
+        if (fields != null) {
+          ...
+        }
+
+    If the reader might be multi-segment, you must do this:
     
-    Fields fields = MultiFields.getFields(reader);
-    if (fields != null) {
-      ...
-    }
+        Fields fields = MultiFields.getFields(reader);
+        if (fields != null) {
+          ...
+        }
   
-  The fields may be null (eg if the reader has no fields).
+    The fields may be null (eg if the reader has no fields).
 
-  Note that the MultiFields approach entails a performance hit on
-  MultiReaders, as it must merge terms/docs/positions on the fly. It's
-  generally better to instead get the sequential readers (use
-  oal.util.ReaderUtil) and then step through those readers yourself,
-  if you can (this is how Lucene drives searches).
+    Note that the MultiFields approach entails a performance hit on
+    MultiReaders, as it must merge terms/docs/positions on the fly. It's
+    generally better to instead get the sequential readers (use
+    oal.util.ReaderUtil) and then step through those readers yourself,
+    if you can (this is how Lucene drives searches).
+
+    If you pass a SegmentReader to MultiFields.fiels it will simply
+    return reader.fields(), so there is no performance hit in that
+    case.
+
+    Once you have a non-null Fields you can do this:
+
+        Terms terms = fields.terms("field");
+        if (terms != null) {
+          ...
+        }
 
-  If you pass a SegmentReader to MultiFields.fiels it will simply
-  return reader.fields(), so there is no performance hit in that
-  case.
+    The terms may be null (eg if the field does not exist).
 
-  Once you have a non-null Fields you can do this:
+    Once you have a non-null terms you can get an enum like this:
 
-    Terms terms = fields.terms("field");
-    if (terms != null) {
-      ...
-    }
+        TermsEnum termsEnum = terms.iterator();
 
-  The terms may be null (eg if the field does not exist).
+    The returned TermsEnum will not be null.
 
-  Once you have a non-null terms you can get an enum like this:
+    You can then .next() through the TermsEnum, or seek.  If you want a
+    DocsEnum, do this:
 
-    TermsEnum termsEnum = terms.iterator();
+        Bits liveDocs = reader.getLiveDocs();
+        DocsEnum docsEnum = null;
 
-  The returned TermsEnum will not be null.
+        docsEnum = termsEnum.docs(liveDocs, docsEnum);
 
-  You can then .next() through the TermsEnum, or seek.  If you want a
-  DocsEnum, do this:
+    You can pass in a prior DocsEnum and it will be reused if possible.
 
-    Bits liveDocs = reader.getLiveDocs();
-    DocsEnum docsEnum = null;
+    Likewise for DocsAndPositionsEnum.
 
-    docsEnum = termsEnum.docs(liveDocs, docsEnum);
+    IndexReader has several sugar methods (which just go through the
+    above steps, under the hood).  Instead of:
 
-  You can pass in a prior DocsEnum and it will be reused if possible.
+        Term t;
+        TermDocs termDocs = reader.termDocs();
+        termDocs.seek(t);
 
-  Likewise for DocsAndPositionsEnum.
+    do this:
 
-  IndexReader has several sugar methods (which just go through the
-  above steps, under the hood).  Instead of:
+        String field;
+        BytesRef text;
+        DocsEnum docsEnum = reader.termDocsEnum(reader.getLiveDocs(), field, text);
 
-    Term t;
-    TermDocs termDocs = reader.termDocs();
-    termDocs.seek(t);
+    Likewise for DocsAndPositionsEnum.
 
-  do this:
+## LUCENE-2380: FieldCache.getStrings/Index --> FieldCache.getDocTerms/Index
 
-    String field;
-    BytesRef text;
-    DocsEnum docsEnum = reader.termDocsEnum(reader.getLiveDocs(), field, text);
+  * The field values returned when sorting by SortField.STRING are now
+    BytesRef.  You can call value.utf8ToString() to convert back to
+    string, if necessary.
 
-  Likewise for DocsAndPositionsEnum.
+  * In FieldCache, getStrings (returning String[]) has been replaced
+    with getTerms (returning a FieldCache.DocTerms instance).
+    DocTerms provides a getTerm method, taking a docID and a BytesRef
+    to fill (which must not be null), and it fills it in with the
+    reference to the bytes for that term.
 
-* LUCENE-2600: remove IndexReader.isDeleted
+    If you had code like this before:
 
-  Instead of IndexReader.isDeleted, do this:
+        String[] values = FieldCache.DEFAULT.getStrings(reader, field);
+        ...
+        String aValue = values[docID];
 
-    import org.apache.lucene.util.Bits;
-    import org.apache.lucene.index.MultiFields;
+    you can do this instead:
 
-    Bits liveDocs = MultiFields.getLiveDocs(indexReader);
-    if (!liveDocs.get(docID)) {
-      // document is deleted...
-    }
-    
-* LUCENE-2858, LUCENE-3733: The abstract class IndexReader has been 
-  refactored to expose only essential methods to access stored fields 
-  during display of search results. It is no longer possible to retrieve 
-  terms or postings data from the underlying index, not even deletions are 
-  visible anymore. You can still pass IndexReader as constructor parameter 
-  to IndexSearcher and execute your searches; Lucene will automatically 
-  delegate procedures like query rewriting and document collection atomic 
-  subreaders. 
-
-  If you want to dive deeper into the index and want to write own queries, 
-  take a closer look at the new abstract sub-classes AtomicReader and 
-  CompositeReader: 
-
-  AtomicReader instances are now the only source of Terms, Postings, 
-  DocValues and FieldCache. Queries are forced to execute on a Atomic 
-  reader on a per-segment basis and FieldCaches are keyed by 
-  AtomicReaders. 
-
-  Its counterpart CompositeReader exposes a utility method to retrieve 
-  its composites. But watch out, composites are not necessarily atomic. 
-  Next to the added type-safety we also removed the notion of 
-  index-commits and version numbers from the abstract IndexReader, the 
-  associations with IndexWriter were pulled into a specialized 
-  DirectoryReader. To open Directory-based indexes use 
-  DirectoryReader.open(), the corresponding method in IndexReader is now 
-  deprecated for easier migration. Only DirectoryReader supports commits, 
-  versions, and reopening with openIfChanged(). Terms, postings, 
-  docvalues, and norms can from now on only be retrieved using 
-  AtomicReader; DirectoryReader and MultiReader extend CompositeReader, 
-  only offering stored fields and access to the sub-readers (which may be 
-  composite or atomic). 
-
-  If you have more advanced code dealing with custom Filters, you might 
-  have noticed another new class hierarchy in Lucene (see LUCENE-2831): 
-  IndexReaderContext with corresponding Atomic-/CompositeReaderContext. 
-
-  The move towards per-segment search Lucene 2.9 exposed lots of custom 
-  Queries and Filters that couldn't handle it. For example, some Filter 
-  implementations expected the IndexReader passed in is identical to the 
-  IndexReader passed to IndexSearcher with all its advantages like 
-  absolute document IDs etc. Obviously this "paradigm-shift" broke lots of 
-  applications and especially those that utilized cross-segment data 
-  structures (like Apache Solr). 
-
-  In Lucene 4.0, we introduce IndexReaderContexts "searcher-private" 
-  reader hierarchy. During Query or Filter execution Lucene no longer 
-  passes raw readers down Queries, Filters or Collectors; instead 
-  components are provided an AtomicReaderContext (essentially a hierarchy 
-  leaf) holding relative properties like the document-basis in relation to 
-  the top-level reader. This allows Queries & Filter to build up logic 
-  based on document IDs, albeit the per-segment orientation. 
-
-  There are still valid use-cases where top-level readers ie. "atomic 
-  views" on the index are desirable. Let say you want to iterate all terms 
-  of a complete index for auto-completion or facetting, Lucene provides 
-  utility wrappers like SlowCompositeReaderWrapper (LUCENE-2597) emulating 
-  an AtomicReader. Note: using "atomicity emulators" can cause serious 
-  slowdowns due to the need to merge terms, postings, DocValues, and 
-  FieldCache, use them with care! 
+        DocTerms values = FieldCache.DEFAULT.getTerms(reader, field);
+        ...
+        BytesRef term = new BytesRef();
+        String aValue = values.getTerm(docID, term).utf8ToString();
 
-* LUCENE-2674: A new idfExplain method was added to Similarity, that
-  accepts an incoming docFreq.  If you subclass Similarity, make sure
-  you also override this method on upgrade, otherwise your
-  customizations won't run for certain MultiTermQuerys.
+    Note however that it can be costly to convert to String, so it's
+    better to work directly with the BytesRef.
 
-* LUCENE-2413: Lucene's core and contrib analyzers, along with Solr's analyzers,
-  were consolidated into lucene/analysis. During the refactoring some
-  package names have changed:
-    - o.a.l.analysis.KeywordAnalyzer -> o.a.l.analysis.core.KeywordAnalyzer
-    - o.a.l.analysis.KeywordTokenizer -> o.a.l.analysis.core.KeywordTokenizer
-    - o.a.l.analysis.LetterTokenizer -> o.a.l.analysis.core.LetterTokenizer
-    - o.a.l.analysis.LowerCaseFilter -> o.a.l.analysis.core.LowerCaseFilter
-    - o.a.l.analysis.LowerCaseTokenizer -> o.a.l.analysis.core.LowerCaseTokenizer
-    - o.a.l.analysis.SimpleAnalyzer -> o.a.l.analysis.core.SimpleAnalyzer
-    - o.a.l.analysis.StopAnalyzer -> o.a.l.analysis.core.StopAnalyzer
-    - o.a.l.analysis.StopFilter -> o.a.l.analysis.core.StopFilter
-    - o.a.l.analysis.WhitespaceAnalyzer -> o.a.l.analysis.core.WhitespaceAnalyzer
-    - o.a.l.analysis.WhitespaceTokenizer -> o.a.l.analysis.core.WhitespaceTokenizer
-    - o.a.l.analysis.PorterStemFilter -> o.a.l.analysis.en.PorterStemFilter
-    - o.a.l.analysis.ASCIIFoldingFilter -> o.a.l.analysis.miscellaneous.ASCIIFoldingFilter
-    - o.a.l.analysis.ISOLatin1AccentFilter -> o.a.l.analysis.miscellaneous.ISOLatin1AccentFilter
-    - o.a.l.analysis.KeywordMarkerFilter -> o.a.l.analysis.miscellaneous.KeywordMarkerFilter
-    - o.a.l.analysis.LengthFilter -> o.a.l.analysis.miscellaneous.LengthFilter
-    - o.a.l.analysis.PerFieldAnalyzerWrapper -> o.a.l.analysis.miscellaneous.PerFieldAnalyzerWrapper
-    - o.a.l.analysis.TeeSinkTokenFilter -> o.a.l.analysis.sinks.TeeSinkTokenFilter
-    - o.a.l.analysis.CharFilter -> o.a.l.analysis.charfilter.CharFilter
-    - o.a.l.analysis.BaseCharFilter -> o.a.l.analysis.charfilter.BaseCharFilter
-    - o.a.l.analysis.MappingCharFilter -> o.a.l.analysis.charfilter.MappingCharFilter
-    - o.a.l.analysis.NormalizeCharMap -> o.a.l.analysis.charfilter.NormalizeCharMap
-    - o.a.l.analysis.CharArraySet -> o.a.l.analysis.util.CharArraySet
-    - o.a.l.analysis.CharArrayMap -> o.a.l.analysis.util.CharArrayMap
-    - o.a.l.analysis.ReusableAnalyzerBase -> o.a.l.analysis.util.ReusableAnalyzerBase
-    - o.a.l.analysis.StopwordAnalyzerBase -> o.a.l.analysis.util.StopwordAnalyzerBase
-    - o.a.l.analysis.WordListLoader -> o.a.l.analysis.util.WordListLoader
-    - o.a.l.analysis.CharTokenizer -> o.a.l.analysis.util.CharTokenizer
-    - o.a.l.util.CharacterUtils -> o.a.l.analysis.util.CharacterUtils
+  * Similarly, in FieldCache, getStringIndex (returning a StringIndex
+    instance, with direct arrays int[] order and String[] lookup) has
+    been replaced with getTermsIndex (returning a
+    FieldCache.DocTermsIndex instance).  DocTermsIndex provides the
+    getOrd(int docID) method to lookup the int order for a document,
+    lookup(int ord, BytesRef reuse) to lookup the term from a given
+    order, and the sugar method getTerm(int docID, BytesRef reuse)
+    which internally calls getOrd and then lookup.
 
-* LUCENE-2514: The option to use a Collator's order (instead of binary order) for
-  sorting and range queries has been moved to lucene/queries.
+    If you had code like this before:
 
-  The Collated TermRangeQuery/Filter has been moved to SlowCollatedTermRangeQuery/Filter, 
-  and the collated sorting has been moved to SlowCollatedStringComparator.
+        StringIndex idx = FieldCache.DEFAULT.getStringIndex(reader, field);
+        ...
+        int ord = idx.order[docID];
+        String aValue = idx.lookup[ord];
 
-  Note: this functionality isn't very scalable and if you are using it, consider 
-  indexing collation keys with the collation support in the analysis module instead.
-  
-  To perform collated range queries, use a suitable collating analyzer: CollationKeyAnalyzer 
-  or ICUCollationKeyAnalyzer, and set qp.setAnalyzeRangeTerms(true).
-  
-  TermRangeQuery and TermRangeFilter now work purely on bytes. Both have helper factory methods
-  (newStringRange) similar to the NumericRange API, to easily perform range queries on Strings.
-  
-* LUCENE-2691: The near-real-time API has moved from IndexWriter to
-  IndexReader.  Instead of IndexWriter.getReader(), call
-  IndexReader.open(IndexWriter) or IndexReader.reopen(IndexWriter).
+    you can do this instead:
 
-* LUCENE-2690: MultiTermQuery boolean rewrites per segment.
-  Also MultiTermQuery.getTermsEnum() now takes an AttributeSource. FuzzyTermsEnum
-  is both consumer and producer of attributes: MTQ.BoostAttribute is
-  added to the FuzzyTermsEnum and MTQ's rewrite mode consumes it.
-  The other way round MTQ.TopTermsBooleanQueryRewrite supplys a
-  global AttributeSource to each segments TermsEnum. The TermsEnum is consumer
-  and gets the current minimum competitive boosts (MTQ.MaxNonCompetitiveBoostAttribute).
+        DocTermsIndex idx = FieldCache.DEFAULT.getTermsIndex(reader, field);
+        ...
+        int ord = idx.getOrd(docID);
+        BytesRef term = new BytesRef();
+        String aValue = idx.lookup(ord, term).utf8ToString();
 
-* LUCENE-2374: The backwards layer in AttributeImpl was removed. To support correct
-  reflection of AttributeImpl instances, where the reflection was done using deprecated
-  toString() parsing, you have to now override reflectWith() to customize output.
-  toString() is no longer implemented by AttributeImpl, so if you have overridden
-  toString(), port your customization over to reflectWith(). reflectAsString() would
-  then return what toString() did before.
+    Note however that it can be costly to convert to String, so it's
+    better to work directly with the BytesRef.
 
-* LUCENE-2236, LUCENE-2912: DefaultSimilarity can no longer be set statically 
-  (and dangerously) for the entire JVM.
-  Similarity can now be configured on a per-field basis (via PerFieldSimilarityWrapper)
-  Similarity has a lower-level API, if you want the higher-level vector-space API
-  like in previous Lucene releases, then look at TFIDFSimilarity.
+    DocTermsIndex also has a getTermsEnum() method, which returns an
+    iterator (TermsEnum) over the term values in the index (ie,
+    iterates ord = 0..numOrd()-1).
 
-* LUCENE-1076: TieredMergePolicy is now the default merge policy.
-  It's able to merge non-contiguous segments; this may cause problems
-  for applications that rely on Lucene's internal document ID
-  assigment.  If so, you should instead use LogByteSize/DocMergePolicy
-  during indexing.
+  * StringComparatorLocale is now more CPU costly than it was before
+    (it was already very CPU costly since it does not compare using
+    indexed collation keys; use CollationKeyFilter for better
+    performance), since it converts BytesRef -> String on the fly.
+    Also, the field values returned when sorting by SortField.STRING
+    are now BytesRef.
 
-* LUCENE-2883: Lucene's o.a.l.search.function ValueSource based functionality, was consolidated
-  into lucene/queries along with Solr's similar functionality.  The following classes were moved:
-   - o.a.l.search.function.CustomScoreQuery -> o.a.l.queries.CustomScoreQuery
-   - o.a.l.search.function.CustomScoreProvider -> o.a.l.queries.CustomScoreProvider
-   - o.a.l.search.function.NumericIndexDocValueSource -> o.a.l.queries.function.valuesource.NumericIndexDocValueSource
-  The following lists the replacement classes for those removed:
-   - o.a.l.search.function.ByteFieldSource -> o.a.l.queries.function.valuesource.ByteFieldSource
-   - o.a.l.search.function.DocValues -> o.a.l.queries.function.DocValues
-   - o.a.l.search.function.FieldCacheSource -> o.a.l.queries.function.valuesource.FieldCacheSource
-   - o.a.l.search.function.FieldScoreQuery ->o.a.l.queries.function.FunctionQuery
-   - o.a.l.search.function.FloatFieldSource -> o.a.l.queries.function.valuesource.FloatFieldSource
-   - o.a.l.search.function.IntFieldSource -> o.a.l.queries.function.valuesource.IntFieldSource
-   - o.a.l.search.function.OrdFieldSource -> o.a.l.queries.function.valuesource.OrdFieldSource
-   - o.a.l.search.function.ReverseOrdFieldSource -> o.a.l.queries.function.valuesource.ReverseOrdFieldSource
-   - o.a.l.search.function.ShortFieldSource -> o.a.l.queries.function.valuesource.ShortFieldSource
-   - o.a.l.search.function.ValueSource -> o.a.l.queries.function.ValueSource
-   - o.a.l.search.function.ValueSourceQuery -> o.a.l.queries.function.FunctionQuery
-
-   DocValues are now named FunctionValues, to not confuse with Lucene's per-document values.
-
-* LUCENE-2392: Enable flexible scoring:
-
-  The existing "Similarity" api is now TFIDFSimilarity, if you were extending
-  Similarity before, you should likely extend this instead.
-
-  Weight.normalize no longer takes a norm value that incorporates the top-level
-  boost from outer queries such as BooleanQuery, instead it takes 2 parameters,
-  the outer boost (topLevelBoost) and the norm. Weight.sumOfSquaredWeights has
-  been renamed to Weight.getValueForNormalization().
+  * FieldComparator.StringOrdValComparator has been renamed to
+    TermOrdValComparator, and now uses BytesRef for its values.
+    Likewise for StringValComparator, renamed to TermValComparator.
+    This means when sorting by SortField.STRING or
+    SortField.STRING_VAL (or directly invoking these comparators) the
+    values returned in the FieldDoc.fields array will be BytesRef not
+    String.  You can call the .utf8ToString() method on the BytesRef
+    instances, if necessary.
 
-  The scorePayload method now takes a BytesRef. It is never null.
+## LUCENE-2600: IndexReaders are now read-only
 
-* LUCENE-3722: Similarity methods and collection/term statistics now take
-  long instead of int (to enable distributed scoring of > 2B docs). 
-  For example, in TFIDFSimilarity idf(int, int) is now idf(long, long). 
+  Instead of IndexReader.isDeleted, do this:
 
-* LUCENE-3559: The methods "docFreq" and "maxDoc" on IndexSearcher were removed,
-  as these are no longer used by the scoring system.
+      import org.apache.lucene.util.Bits;
+      import org.apache.lucene.index.MultiFields;
 
-  If you were using these casually in your code for reasons unrelated to scoring,
-  call them on the IndexSearcher's reader instead: getIndexReader().
+      Bits liveDocs = MultiFields.getLiveDocs(indexReader);
+      if (!liveDocs.get(docID)) {
+        // document is deleted...
+      }
+    
+## LUCENE-2858, LUCENE-3733: IndexReader --> AtomicReader/CompositeReader/DirectoryReader refactoring
 
-  If you were subclassing IndexSearcher and overriding these methods to alter
-  scoring, override IndexSearcher's termStatistics() and collectionStatistics()
-  methods instead.
+The abstract class IndexReader has been 
+refactored to expose only essential methods to access stored fields 
+during display of search results. It is no longer possible to retrieve 
+terms or postings data from the underlying index, not even deletions are 
+visible anymore. You can still pass IndexReader as constructor parameter 
+to IndexSearcher and execute your searches; Lucene will automatically 
+delegate procedures like query rewriting and document collection atomic 
+subreaders. 
+
+If you want to dive deeper into the index and want to write own queries, 
+take a closer look at the new abstract sub-classes AtomicReader and 
+CompositeReader: 
+
+AtomicReader instances are now the only source of Terms, Postings, 
+DocValues and FieldCache. Queries are forced to execute on a Atomic 
+reader on a per-segment basis and FieldCaches are keyed by 
+AtomicReaders. 
+
+Its counterpart CompositeReader exposes a utility method to retrieve 
+its composites. But watch out, composites are not necessarily atomic. 
+Next to the added type-safety we also removed the notion of 
+index-commits and version numbers from the abstract IndexReader, the 
+associations with IndexWriter were pulled into a specialized 
+DirectoryReader. To open Directory-based indexes use 
+DirectoryReader.open(), the corresponding method in IndexReader is now 
+deprecated for easier migration. Only DirectoryReader supports commits, 
+versions, and reopening with openIfChanged(). Terms, postings, 
+docvalues, and norms can from now on only be retrieved using 
+AtomicReader; DirectoryReader and MultiReader extend CompositeReader, 
+only offering stored fields and access to the sub-readers (which may be 
+composite or atomic). 
+
+If you have more advanced code dealing with custom Filters, you might 
+have noticed another new class hierarchy in Lucene (see LUCENE-2831): 
+IndexReaderContext with corresponding Atomic-/CompositeReaderContext. 
+
+The move towards per-segment search Lucene 2.9 exposed lots of custom 
+Queries and Filters that couldn't handle it. For example, some Filter 
+implementations expected the IndexReader passed in is identical to the 
+IndexReader passed to IndexSearcher with all its advantages like 
+absolute document IDs etc. Obviously this "paradigm-shift" broke lots of 
+applications and especially those that utilized cross-segment data 
+structures (like Apache Solr). 
+
+In Lucene 4.0, we introduce IndexReaderContexts "searcher-private" 
+reader hierarchy. During Query or Filter execution Lucene no longer 
+passes raw readers down Queries, Filters or Collectors; instead 
+components are provided an AtomicReaderContext (essentially a hierarchy 
+leaf) holding relative properties like the document-basis in relation to 
+the top-level reader. This allows Queries & Filter to build up logic 
+based on document IDs, albeit the per-segment orientation. 
+
+There are still valid use-cases where top-level readers ie. "atomic 
+views" on the index are desirable. Let say you want to iterate all terms 
+of a complete index for auto-completion or facetting, Lucene provides 
+utility wrappers like SlowCompositeReaderWrapper (LUCENE-2597) emulating 
+an AtomicReader. Note: using "atomicity emulators" can cause serious 
+slowdowns due to the need to merge terms, postings, DocValues, and 
+FieldCache, use them with care! 
+
+## LUCENE-2413: Analyzer package changes
+
+Lucene's core and contrib analyzers, along with Solr's analyzers,
+were consolidated into lucene/analysis. During the refactoring some
+package names have changed:
+
+  - o.a.l.analysis.KeywordAnalyzer -> o.a.l.analysis.core.KeywordAnalyzer
+  - o.a.l.analysis.KeywordTokenizer -> o.a.l.analysis.core.KeywordTokenizer
+  - o.a.l.analysis.LetterTokenizer -> o.a.l.analysis.core.LetterTokenizer
+  - o.a.l.analysis.LowerCaseFilter -> o.a.l.analysis.core.LowerCaseFilter
+  - o.a.l.analysis.LowerCaseTokenizer -> o.a.l.analysis.core.LowerCaseTokenizer
+  - o.a.l.analysis.SimpleAnalyzer -> o.a.l.analysis.core.SimpleAnalyzer
+  - o.a.l.analysis.StopAnalyzer -> o.a.l.analysis.core.StopAnalyzer
+  - o.a.l.analysis.StopFilter -> o.a.l.analysis.core.StopFilter
+  - o.a.l.analysis.WhitespaceAnalyzer -> o.a.l.analysis.core.WhitespaceAnalyzer
+  - o.a.l.analysis.WhitespaceTokenizer -> o.a.l.analysis.core.WhitespaceTokenizer
+  - o.a.l.analysis.PorterStemFilter -> o.a.l.analysis.en.PorterStemFilter
+  - o.a.l.analysis.ASCIIFoldingFilter -> o.a.l.analysis.miscellaneous.ASCIIFoldingFilter
+  - o.a.l.analysis.ISOLatin1AccentFilter -> o.a.l.analysis.miscellaneous.ISOLatin1AccentFilter
+  - o.a.l.analysis.KeywordMarkerFilter -> o.a.l.analysis.miscellaneous.KeywordMarkerFilter
+  - o.a.l.analysis.LengthFilter -> o.a.l.analysis.miscellaneous.LengthFilter
+  - o.a.l.analysis.PerFieldAnalyzerWrapper -> o.a.l.analysis.miscellaneous.PerFieldAnalyzerWrapper
+  - o.a.l.analysis.TeeSinkTokenFilter -> o.a.l.analysis.sinks.TeeSinkTokenFilter
+  - o.a.l.analysis.CharFilter -> o.a.l.analysis.charfilter.CharFilter
+  - o.a.l.analysis.BaseCharFilter -> o.a.l.analysis.charfilter.BaseCharFilter
+  - o.a.l.analysis.MappingCharFilter -> o.a.l.analysis.charfilter.MappingCharFilter
+  - o.a.l.analysis.NormalizeCharMap -> o.a.l.analysis.charfilter.NormalizeCharMap
+  - o.a.l.analysis.CharArraySet -> o.a.l.analysis.util.CharArraySet
+  - o.a.l.analysis.CharArrayMap -> o.a.l.analysis.util.CharArrayMap
+  - o.a.l.analysis.ReusableAnalyzerBase -> o.a.l.analysis.util.ReusableAnalyzerBase
+  - o.a.l.analysis.StopwordAnalyzerBase -> o.a.l.analysis.util.StopwordAnalyzerBase
+  - o.a.l.analysis.WordListLoader -> o.a.l.analysis.util.WordListLoader
+  - o.a.l.analysis.CharTokenizer -> o.a.l.analysis.util.CharTokenizer
+  - o.a.l.util.CharacterUtils -> o.a.l.analysis.util.CharacterUtils
+
+## LUCENE-2514: Collators
+
+The option to use a Collator's order (instead of binary order) for
+sorting and range queries has been moved to lucene/queries.
+The Collated TermRangeQuery/Filter has been moved to SlowCollatedTermRangeQuery/Filter, 
+and the collated sorting has been moved to SlowCollatedStringComparator.
+
+Note: this functionality isn't very scalable and if you are using it, consider 
+indexing collation keys with the collation support in the analysis module instead.
+
+To perform collated range queries, use a suitable collating analyzer: CollationKeyAnalyzer 
+or ICUCollationKeyAnalyzer, and set qp.setAnalyzeRangeTerms(true).
+
+TermRangeQuery and TermRangeFilter now work purely on bytes. Both have helper factory methods
+(newStringRange) similar to the NumericRange API, to easily perform range queries on Strings.
+
+## LUCENE-2883: ValueSource changes
+
+Lucene's o.a.l.search.function ValueSource based functionality, was consolidated
+into lucene/queries along with Solr's similar functionality.  The following classes were moved:
+
+ - o.a.l.search.function.CustomScoreQuery -> o.a.l.queries.CustomScoreQuery
+ - o.a.l.search.function.CustomScoreProvider -> o.a.l.queries.CustomScoreProvider
+ - o.a.l.search.function.NumericIndexDocValueSource -> o.a.l.queries.function.valuesource.NumericIndexDocValueSource
+
+The following lists the replacement classes for those removed:
+
+ - o.a.l.search.function.ByteFieldSource -> o.a.l.queries.function.valuesource.ByteFieldSource
+ - o.a.l.search.function.DocValues -> o.a.l.queries.function.DocValues
+ - o.a.l.search.function.FieldCacheSource -> o.a.l.queries.function.valuesource.FieldCacheSource
+ - o.a.l.search.function.FieldScoreQuery ->o.a.l.queries.function.FunctionQuery
+ - o.a.l.search.function.FloatFieldSource -> o.a.l.queries.function.valuesource.FloatFieldSource
+ - o.a.l.search.function.IntFieldSource -> o.a.l.queries.function.valuesource.IntFieldSource
+ - o.a.l.search.function.OrdFieldSource -> o.a.l.queries.function.valuesource.OrdFieldSource
+ - o.a.l.search.function.ReverseOrdFieldSource -> o.a.l.queries.function.valuesource.ReverseOrdFieldSource
+ - o.a.l.search.function.ShortFieldSource -> o.a.l.queries.function.valuesource.ShortFieldSource
+ - o.a.l.search.function.ValueSource -> o.a.l.queries.function.ValueSource
+ - o.a.l.search.function.ValueSourceQuery -> o.a.l.queries.function.FunctionQuery
+
+DocValues are now named FunctionValues, to not confuse with Lucene's per-document values.
+
+## LUCENE-2392: Enable flexible scoring
+
+The existing "Similarity" api is now TFIDFSimilarity, if you were extending
+Similarity before, you should likely extend this instead.
+
+Weight.normalize no longer takes a norm value that incorporates the top-level
+boost from outer queries such as BooleanQuery, instead it takes 2 parameters,
+the outer boost (topLevelBoost) and the norm. Weight.sumOfSquaredWeights has
+been renamed to Weight.getValueForNormalization().
+
+The scorePayload method now takes a BytesRef. It is never null.
+
+## LUCENE-3283: Query parsers moved to separate module
+
+Lucene's core o.a.l.queryParser QueryParsers have been consolidated into lucene/queryparser,
+where other QueryParsers from the codebase will also be placed.  The following classes were moved:
 
-* LUCENE-3283: Lucene's core o.a.l.queryParser QueryParsers have been consolidated into lucene/queryparser,
-  where other QueryParsers from the codebase will also be placed.  The following classes were moved:
   - o.a.l.queryParser.CharStream -> o.a.l.queryparser.classic.CharStream
   - o.a.l.queryParser.FastCharStream -> o.a.l.queryparser.classic.FastCharStream
   - o.a.l.queryParser.MultiFieldQueryParser -> o.a.l.queryparser.classic.MultiFieldQueryParser
@@ -480,9 +444,7 @@ LUCENE-1458, LUCENE-2111: Flexible Index
   - o.a.l.queryParser.QueryParserToken -> o.a.l.queryparser.classic.Token
   - o.a.l.queryParser.QueryParserTokenMgrError -> o.a.l.queryparser.classic.TokenMgrError
 
-
-
-* LUCENE-2308,LUCENE-3453: Separate IndexableFieldType from Field instances
+## LUCENE-2308, LUCENE-3453: Separate IndexableFieldType from Field instances
 
 With this change, the indexing details (indexed, tokenized, norms,
 indexOptions, stored, etc.) are moved into a separate FieldType
@@ -498,15 +460,11 @@ Certain field types are pre-defined sinc
     not tokenize).  This field turns off norms and indexes only doc
     IDS (does not index term frequency nor positions).  This field
     does not store its value, but exposes TYPE_STORED as well.
-
   * TextField: indexes and tokenizes a String, Reader or TokenStream
     value, without term vectors.  This field does not store its value,
     but exposes TYPE_STORED as well.
-
   * StoredField: field that stores its value
-
   * DocValuesField: indexes the value as a DocValues field
-
   * NumericField: indexes the numeric value so that NumericRangeQuery
     can be used at search-time.
 
@@ -515,23 +473,22 @@ instantiate the above class.  If you nee
 add a separate StoredField to the document, or you can use
 TYPE_STORED for the field:
 
-  Field f = new Field("field", "value", StringField.TYPE_STORED);
+    Field f = new Field("field", "value", StringField.TYPE_STORED);
 
 Alternatively, if an existing type is close to what you want but you
 need to make a few changes, you can copy that type and make changes:
 
-  FieldType bodyType = new FieldType(TextField.TYPE_STORED);
-  bodyType.setStoreTermVectors(true);
-
+    FieldType bodyType = new FieldType(TextField.TYPE_STORED);
+    bodyType.setStoreTermVectors(true);
 
 You can of course also create your own FieldType from scratch:
 
-  FieldType t = new FieldType();
-  t.setIndexed(true);
-  t.setStored(true);
-  t.setOmitNorms(true);
-  t.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
-  t.freeze();
+    FieldType t = new FieldType();
+    t.setIndexed(true);
+    t.setStored(true);
+    t.setOmitNorms(true);
+    t.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+    t.freeze();
 
 FieldType has a freeze() method to prevent further changes.
 
@@ -541,65 +498,116 @@ enums.
 
 When migrating from the 3.x API, if you did this before:
 
-  new Field("field", "value", Field.Store.NO, Field.Indexed.NOT_ANALYZED_NO_NORMS)
+    new Field("field", "value", Field.Store.NO, Field.Indexed.NOT_ANALYZED_NO_NORMS)
 
 you can now do this:
 
-  new StringField("field", "value")
+    new StringField("field", "value")
 
 (though note that StringField indexes DOCS_ONLY).
 
 If instead the value was stored:
 
-  new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED_NO_NORMS)
+    new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED_NO_NORMS)
 
 you can now do this:
 
-  new Field("field", "value", StringField.TYPE_STORED)
+    new Field("field", "value", StringField.TYPE_STORED)
 
 If you didn't omit norms:
 
-  new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED)
+    new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED)
 
 you can now do this:
 
-  FieldType ft = new FieldType(StringField.TYPE_STORED);
-  ft.setOmitNorms(false);
-  new Field("field", "value", ft)
+    FieldType ft = new FieldType(StringField.TYPE_STORED);
+    ft.setOmitNorms(false);
+    new Field("field", "value", ft)
 
 If you did this before (value can be String or Reader):
 
-  new Field("field", value, Field.Store.NO, Field.Indexed.ANALYZED)
+    new Field("field", value, Field.Store.NO, Field.Indexed.ANALYZED)
 
 you can now do this:
 
-  new TextField("field", value)
+    new TextField("field", value)
 
 If instead the value was stored:
 
-  new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED)
+    new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED)
 
 you can now do this:
 
-  new Field("field", value, TextField.TYPE_STORED)
+    new Field("field", value, TextField.TYPE_STORED)
 
 If in addition you omit norms:
 
-  new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED_NO_NORMS)
+    new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED_NO_NORMS)
 
 you can now do this:
 
-  FieldType ft = new FieldType(TextField.TYPE_STORED);
-  ft.setOmitNorms(true);
-  new Field("field", value, ft)
+    FieldType ft = new FieldType(TextField.TYPE_STORED);
+    ft.setOmitNorms(true);
+    new Field("field", value, ft)
 
 If you did this before (bytes is a byte[]):
 
-  new Field("field", bytes)
+    new Field("field", bytes)
 
 you can now do this:
 
-  new StoredField("field", bytes)
+    new StoredField("field", bytes)
+
+## Other changes
+
+* LUCENE-2674:
+  A new idfExplain method was added to Similarity, that
+  accepts an incoming docFreq.  If you subclass Similarity, make sure
+  you also override this method on upgrade, otherwise your
+  customizations won't run for certain MultiTermQuerys.
+
+* LUCENE-2691: The near-real-time API has moved from IndexWriter to
+  IndexReader.  Instead of IndexWriter.getReader(), call
+  IndexReader.open(IndexWriter) or IndexReader.reopen(IndexWriter).
+
+* LUCENE-2690: MultiTermQuery boolean rewrites per segment.
+  Also MultiTermQuery.getTermsEnum() now takes an AttributeSource. FuzzyTermsEnum
+  is both consumer and producer of attributes: MTQ.BoostAttribute is
+  added to the FuzzyTermsEnum and MTQ's rewrite mode consumes it.
+  The other way round MTQ.TopTermsBooleanQueryRewrite supplys a
+  global AttributeSource to each segments TermsEnum. The TermsEnum is consumer
+  and gets the current minimum competitive boosts (MTQ.MaxNonCompetitiveBoostAttribute).
+
+* LUCENE-2374: The backwards layer in AttributeImpl was removed. To support correct
+  reflection of AttributeImpl instances, where the reflection was done using deprecated
+  toString() parsing, you have to now override reflectWith() to customize output.
+  toString() is no longer implemented by AttributeImpl, so if you have overridden
+  toString(), port your customization over to reflectWith(). reflectAsString() would
+  then return what toString() did before.
+
+* LUCENE-2236, LUCENE-2912: DefaultSimilarity can no longer be set statically 
+  (and dangerously) for the entire JVM.
+  Similarity can now be configured on a per-field basis (via PerFieldSimilarityWrapper)
+  Similarity has a lower-level API, if you want the higher-level vector-space API
+  like in previous Lucene releases, then look at TFIDFSimilarity.
+
+* LUCENE-1076: TieredMergePolicy is now the default merge policy.
+  It's able to merge non-contiguous segments; this may cause problems
+  for applications that rely on Lucene's internal document ID
+  assigment.  If so, you should instead use LogByteSize/DocMergePolicy
+  during indexing.
+
+* LUCENE-3722: Similarity methods and collection/term statistics now take
+  long instead of int (to enable distributed scoring of > 2B docs). 
+  For example, in TFIDFSimilarity idf(int, int) is now idf(long, long). 
+
+* LUCENE-3559: The methods "docFreq" and "maxDoc" on IndexSearcher were removed,
+  as these are no longer used by the scoring system.
+  If you were using these casually in your code for reasons unrelated to scoring,
+  call them on the IndexSearcher's reader instead: getIndexReader().
+  If you were subclassing IndexSearcher and overriding these methods to alter
+  scoring, override IndexSearcher's termStatistics() and collectionStatistics()
+  methods instead.
 
 * LUCENE-3396: Analyzer.tokenStream() and .reusableTokenStream() have been made final.
   It is now necessary to use Analyzer.TokenStreamComponents to define an analysis process.
@@ -616,7 +624,6 @@ you can now do this:
   set integer, float and byte values if a single byte is not sufficient.
 
 * LUCENE-2621: Term vectors are now accessed via flexible indexing API.
-
   If you used IndexReader.getTermFreqVector/s before, you should now
   use IndexReader.getTermVectors.  The new method returns a Fields
   instance exposing the inverted index of the one document.  From

Modified: lucene/dev/trunk/lucene/README.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/README.txt?rev=1328978&r1=1328977&r2=1328978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/README.txt (original)
+++ lucene/dev/trunk/lucene/README.txt Sun Apr 22 21:15:27 2012
@@ -1,52 +1,21 @@
-Apache Lucene README file
+# Apache Lucene README file
 
-INTRODUCTION
+## Introduction
 
 Lucene is a Java full-text search engine.  Lucene is not a complete
 application, but rather a code library and API that can easily be used
 to add search capabilities to applications.
 
-The Lucene web site is at:
-  http://lucene.apache.org/
+ * The Lucene web site is at: http://lucene.apache.org/
+ * Please join the Lucene-User mailing list by sending a message to:
+   java-user-subscribe@lucene.apache.org
 
-Please join the Lucene-User mailing list by sending a message to:
-  java-user-subscribe@lucene.apache.org
-
-Files in a binary distribution:
+## Files in a binary distribution
 
 Files are organized by module, for example in core/:
 
-core/lucene-core-XX.jar
+* `core/lucene-core-XX.jar`:
   The compiled core Lucene library.
 
-Additional modules contain the same structure:
-
-analysis/common/: Analyzers for indexing content in different languages and domains
-analysis/icu/: Analysis integration with ICU (International Components for Unicode)
-analysis/kuromoji/: Analyzer for indexing Japanese
-analysis/morfologik/: Analyzer for indexing Polish
-analysis/phonetic/: Analyzer for indexing phonetic signatures (for sounds-alike search)
-analysis/smartcn/: Analyzer for indexing Chinese
-analysis/stempel/: Analyzer for indexing Polish
-analysis/uima/: Analysis integration with Apache UIMA
-benchmark/: System for benchmarking Lucene
-demo/: Simple example code
-facet/: Faceted indexing and search capabilities
-grouping/: Search result grouping
-highlighter/: Highlights search keywords in results
-join/: Index-time and Query-time joins for normalized content
-memory/: Single-document in memory index implementation
-misc/: Index tools and other miscellaneous code
-queries/: Filters and Queries that add to core Lucene
-queryparser/: Query parsers and parsing framework
-sandbox/: Various third party contributions and new ideas.
-spatial/: Geospatial search
-suggest/: Auto-suggest and Spellchecking support
-test-framework/:  Test Framework for testing Lucene-based applications
-  
-docs/index.html
-  The contents of the Lucene website.
-
-docs/api/index.html
-  The Javadoc Lucene API documentation.  This includes the core library, 
-  the test framework, and the demo, as well as all other modules.
+To review the documentation, read the main documentation page, located at:
+`docs/index.html`

Modified: lucene/dev/trunk/lucene/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/build.xml?rev=1328978&r1=1328977&r2=1328978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/build.xml (original)
+++ lucene/dev/trunk/lucene/build.xml Sun Apr 22 21:15:27 2012
@@ -184,11 +184,11 @@
   </target>
 
   <target name="documentation" description="Generate all documentation"
-    depends="javadocs,changes-to-html,doc-index"/>
+    depends="javadocs,changes-to-html,process-webpages"/>
   <target name="javadoc" depends="javadocs"/>
   <target name="javadocs" description="Generate javadoc" depends="javadocs-lucene-core, javadocs-modules, javadocs-test-framework"/>
   
-  <target name="doc-index">
+  <target name="process-webpages" depends="resolve-pegdown">
     <pathconvert pathsep="|" dirsep="/" property="buildfiles">
       <fileset dir="." includes="**/build.xml" excludes="build.xml,analysis/*,build/**,tools/**,backwards/**,site/**"/>
     </pathconvert>
@@ -205,6 +205,12 @@
       <param name="buildfiles" expression="${buildfiles}"/>
       <param name="version" expression="${version}"/>
     </xslt>
+    
+    <pegdown todir="${javadoc.dir}">
+      <fileset dir="." includes="MIGRATE.txt,JRE_VERSION_MIGRATION.txt"/>
+      <globmapper from="*.txt" to="*.html"/>
+    </pegdown>
+
     <copy todir="${javadoc.dir}">
       <fileset dir="site/html" includes="**/*"/>
     </copy>

Modified: lucene/dev/trunk/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/common-build.xml?rev=1328978&r1=1328977&r2=1328978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/common-build.xml (original)
+++ lucene/dev/trunk/lucene/common-build.xml Sun Apr 22 21:15:27 2012
@@ -1506,4 +1506,60 @@ ${tests-output}/junit4-*.suites     - pe
       </scp>
     </sequential>
   </macrodef>
+  
+  <!-- PEGDOWN macro: Before using depend on the target "resolve-pegdown" -->
+  
+  <target name="resolve-pegdown" unless="pegdown.loaded">
+    <ivy:cachepath organisation="org.pegdown" module="pegdown" revision="1.1.0"
+      inline="true" conf="default" type="jar" transitive="true" pathid="pegdown.classpath"/>
+    <property name="pegdown.loaded" value="true"/>
+  </target>
+  
+  <macrodef name="pegdown">
+    <attribute name="todir"/>
+    <attribute name="flatten" default="false"/>
+    <attribute name="overwrite" default="false"/>
+    <element name="nested" optional="false" implicit="true"/>
+    <sequential>
+      <copy todir="@{todir}" flatten="@{flatten}" overwrite="@{overwrite}" verbose="true"
+        preservelastmodified="false" encoding="UTF-8" outputencoding="UTF-8"
+      >
+        <filterchain>
+          <tokenfilter>
+            <filetokenizer/>
+            <replaceregex pattern="\b(LUCENE|SOLR)\-\d+\b" replace="[\0](https://issues.apache.org/jira/browse/\0)" flags="gs"/>
+            <scriptfilter language="javascript" classpathref="pegdown.classpath"><![CDATA[
+              importClass(java.lang.StringBuilder);
+              importClass(org.pegdown.PegDownProcessor);
+              importClass(org.pegdown.Extensions);
+              importClass(org.pegdown.FastEncoder);
+              var markdownSource = self.getToken();
+              var title = undefined;
+              if (markdownSource.search(/^(#+\s*)?(.+)[\n\r]/) == 0) {
+                title = RegExp.$2;
+                // Convert the first line into a markdown heading, if it is not already:
+                if (RegExp.$1 == '') {
+                  markdownSource = '# ' + markdownSource;
+                }
+              }
+              var processor = new PegDownProcessor(
+                Extensions.ABBREVIATIONS | Extensions.AUTOLINKS |
+                Extensions.FENCED_CODE_BLOCKS | Extensions.SMARTS
+              );
+              var html = new StringBuilder('<html>\n<head>\n');
+              if (title) {
+                html.append('<title>').append(FastEncoder.encode(title)).append('</title>\n');
+              }
+              html.append('<meta http-equiv="Content-Type" content="text/html; charset=UTF-8">\n')
+                .append('</head>\n<body>\n')
+                .append(processor.markdownToHtml(markdownSource))
+                .append('\n</body>\n</html>\n');
+              self.setToken(html.toString());
+            ]]></scriptfilter>
+          </tokenfilter>
+        </filterchain>
+        <nested/>
+      </copy>
+    </sequential>
+  </macrodef>
 </project>

Modified: lucene/dev/trunk/lucene/site/xsl/index.xsl
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/site/xsl/index.xsl?rev=1328978&r1=1328977&r2=1328978&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/site/xsl/index.xsl (original)
+++ lucene/dev/trunk/lucene/site/xsl/index.xsl Sun Apr 22 21:15:27 2012
@@ -37,11 +37,14 @@
       <body>
         <div><img src="lucene_green_300.gif"/></div>
         <h1><xsl:text>Apache Lucene </xsl:text><xsl:value-of select="$version"/><xsl:text> Documentation</xsl:text></h1>
+        <p>Lucene is a Java full-text search engine. Lucene is not a complete application, 
+        but rather a code library and API that can easily be used to add search capabilities
+        to applications.</p>
         <p>
           This is the official documentation for <b><xsl:text>Apache Lucene </xsl:text>
           <xsl:value-of select="$version"/></b>. Additional documentation is available in the
           <a href="http://wiki.apache.org/lucene-java">Wiki</a>.
-        </p>
+        </p>        
         <h2>Getting Started</h2>
         <p>The following section is intended as a "getting started" guide. It has three
         audiences: first-time users looking to install Apache Lucene in their
@@ -60,6 +63,8 @@
         <h2>Reference Documents</h2>
           <ul>
             <li><a href="changes/Changes.html">Changes</a>: List of changes in this release.</li>
+            <li><a href="MIGRATE.html">Migration Guide</a>: What changed in Lucene 4; how to migrate code from Lucene 3.x.</li>
+            <li><a href="JRE_VERSION_MIGRATION.html">JRE Version Migration</a>: Information about upgrading between major JRE versions.</li>
             <li><a href="fileformats.html">File Formats</a>: Guide to the index format used by Lucene.</li>
             <li><a href="core/org/apache/lucene/search/package-summary.html#package_description">Search and Scoring in Lucene</a>: Introduction to how Lucene scores documents.</li>
             <li><a href="core/org/apache/lucene/search/similarities/TFIDFSimilarity.html">Classic Scoring Formula</a>: Formula of Lucene's classic <a href="http://en.wikipedia.org/wiki/Vector_Space_Model">Vector Space</a> implementation. (look <a href="core/org/apache/lucene/search/similarities/package-summary.html#package_description">here</a> for other models)</li>



Re: svn commit: r1328978 - in /lucene/dev/trunk/lucene: JRE_VERSION_MIGRATION.txt MIGRATE.txt README.txt build.xml common-build.xml site/xsl/index.xsl

Posted by Robert Muir <rc...@gmail.com>.
This looks really nice in the index.html, thanks for cleaning up these
files too Uwe!

On Sun, Apr 22, 2012 at 5:15 PM,  <us...@apache.org> wrote:
> Author: uschindler
> Date: Sun Apr 22 21:15:27 2012
> New Revision: 1328978
>
> URL: http://svn.apache.org/viewvc?rev=1328978&view=rev
> Log:
> LUCENE-4008: Use pegdown to transform MIGRATE.txt and other text-only files to readable HTML. Please alsows run ant documentation when you have changed anything on those files to check output.
>
> Modified:
>    lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt
>    lucene/dev/trunk/lucene/MIGRATE.txt
>    lucene/dev/trunk/lucene/README.txt
>    lucene/dev/trunk/lucene/build.xml
>    lucene/dev/trunk/lucene/common-build.xml
>    lucene/dev/trunk/lucene/site/xsl/index.xsl
>
> Modified: lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt?rev=1328978&r1=1328977&r2=1328978&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt (original)
> +++ lucene/dev/trunk/lucene/JRE_VERSION_MIGRATION.txt Sun Apr 22 21:15:27 2012
> @@ -1,36 +1,37 @@
> +# JRE Version Migration Guide
> +
>  If possible, use the same JRE major version at both index and search time.
>  When upgrading to a different JRE major version, consider re-indexing.
>
>  Different JRE major versions may implement different versions of Unicode,
>  which will change the way some parts of Lucene treat your text.
>
> -For example: with Java 1.4, LetterTokenizer will split around the character U+02C6,
> +For example: with Java 1.4, `LetterTokenizer` will split around the character U+02C6,
>  but with Java 5 it will not.
>  This is because Java 1.4 implements Unicode 3, but Java 5 implements Unicode 4.
>
>  For reference, JRE major versions with their corresponding Unicode versions:
> -Java 1.4, Unicode 3.0
> -Java 5, Unicode 4.0
> -Java 6, Unicode 4.0
> -Java 7, Unicode 6.0
> +
> + * Java 1.4, Unicode 3.0
> + * Java 5, Unicode 4.0
> + * Java 6, Unicode 4.0
> + * Java 7, Unicode 6.0
>
>  In general, whether or not you need to re-index largely depends upon the data that
>  you are searching, and what was changed in any given Unicode version. For example,
>  if you are completely sure that your content is limited to the "Basic Latin" range
>  of Unicode, you can safely ignore this.
>
> -Special Notes:
> -
> -LUCENE 2.9 TO 3.0, JAVA 1.4 TO JAVA 5 TRANSITION
> +## Special Notes: LUCENE 2.9 TO 3.0, JAVA 1.4 TO JAVA 5 TRANSITION
>
> -* StandardAnalyzer will return the same results under Java 5 as it did under
> +* `StandardAnalyzer` will return the same results under Java 5 as it did under
>  Java 1.4. This is because it is largely independent of the runtime JRE for
>  Unicode support, (with the exception of lowercasing).  However, no changes to
>  casing have occurred in Unicode 4.0 that affect StandardAnalyzer, so if you are
>  using this Analyzer you are NOT affected.
>
> -* SimpleAnalyzer, StopAnalyzer, LetterTokenizer, LowerCaseFilter, and
> -LowerCaseTokenizer may return different results, along with many other Analyzers
> -and TokenStreams in Lucene's analysis modules. If you are using one of these
> +* `SimpleAnalyzer`, `StopAnalyzer`, `LetterTokenizer`, `LowerCaseFilter`, and
> +`LowerCaseTokenizer` may return different results, along with many other `Analyzer`s
> +and `TokenStream`s in Lucene's analysis modules. If you are using one of these
>  components, you may be affected.
>
>
> Modified: lucene/dev/trunk/lucene/MIGRATE.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/MIGRATE.txt?rev=1328978&r1=1328977&r2=1328978&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/MIGRATE.txt (original)
> +++ lucene/dev/trunk/lucene/MIGRATE.txt Sun Apr 22 21:15:27 2012
> @@ -1,138 +1,63 @@
> +# Apache Lucene Migration Guide
>
> -LUCENE-2380: FieldCache.getStrings/Index --> FieldCache.getDocTerms/Index
> +## Four-dimensional enumerations
>
> -  * The field values returned when sorting by SortField.STRING are now
> -    BytesRef.  You can call value.utf8ToString() to convert back to
> -    string, if necessary.
> -
> -  * In FieldCache, getStrings (returning String[]) has been replaced
> -    with getTerms (returning a FieldCache.DocTerms instance).
> -    DocTerms provides a getTerm method, taking a docID and a BytesRef
> -    to fill (which must not be null), and it fills it in with the
> -    reference to the bytes for that term.
> -
> -    If you had code like this before:
> -
> -      String[] values = FieldCache.DEFAULT.getStrings(reader, field);
> -      ...
> -      String aValue = values[docID];
> -
> -    you can do this instead:
> -
> -      DocTerms values = FieldCache.DEFAULT.getTerms(reader, field);
> -      ...
> -      BytesRef term = new BytesRef();
> -      String aValue = values.getTerm(docID, term).utf8ToString();
> -
> -    Note however that it can be costly to convert to String, so it's
> -    better to work directly with the BytesRef.
> -
> -  * Similarly, in FieldCache, getStringIndex (returning a StringIndex
> -    instance, with direct arrays int[] order and String[] lookup) has
> -    been replaced with getTermsIndex (returning a
> -    FieldCache.DocTermsIndex instance).  DocTermsIndex provides the
> -    getOrd(int docID) method to lookup the int order for a document,
> -    lookup(int ord, BytesRef reuse) to lookup the term from a given
> -    order, and the sugar method getTerm(int docID, BytesRef reuse)
> -    which internally calls getOrd and then lookup.
> -
> -    If you had code like this before:
> -
> -      StringIndex idx = FieldCache.DEFAULT.getStringIndex(reader, field);
> -      ...
> -      int ord = idx.order[docID];
> -      String aValue = idx.lookup[ord];
> -
> -    you can do this instead:
> -
> -      DocTermsIndex idx = FieldCache.DEFAULT.getTermsIndex(reader, field);
> -      ...
> -      int ord = idx.getOrd(docID);
> -      BytesRef term = new BytesRef();
> -      String aValue = idx.lookup(ord, term).utf8ToString();
> -
> -    Note however that it can be costly to convert to String, so it's
> -    better to work directly with the BytesRef.
> -
> -    DocTermsIndex also has a getTermsEnum() method, which returns an
> -    iterator (TermsEnum) over the term values in the index (ie,
> -    iterates ord = 0..numOrd()-1).
> -
> -  * StringComparatorLocale is now more CPU costly than it was before
> -    (it was already very CPU costly since it does not compare using
> -    indexed collation keys; use CollationKeyFilter for better
> -    performance), since it converts BytesRef -> String on the fly.
> -    Also, the field values returned when sorting by SortField.STRING
> -    are now BytesRef.
> -
> -  * FieldComparator.StringOrdValComparator has been renamed to
> -    TermOrdValComparator, and now uses BytesRef for its values.
> -    Likewise for StringValComparator, renamed to TermValComparator.
> -    This means when sorting by SortField.STRING or
> -    SortField.STRING_VAL (or directly invoking these comparators) the
> -    values returned in the FieldDoc.fields array will be BytesRef not
> -    String.  You can call the .utf8ToString() method on the BytesRef
> -    instances, if necessary.
> +Flexible indexing changed the low level fields/terms/docs/positions
> +enumeration APIs.  Here are the major changes:
>
> +  * Terms are now binary in nature (arbitrary byte[]), represented
> +    by the BytesRef class (which provides an offset + length "slice"
> +    into an existing byte[]).
>
> -
> -LUCENE-1458, LUCENE-2111: Flexible Indexing
> -
> -  Flexible indexing changed the low level fields/terms/docs/positions
> -  enumeration APIs.  Here are the major changes:
> -
> -    * Terms are now binary in nature (arbitrary byte[]), represented
> -      by the BytesRef class (which provides an offset + length "slice"
> -      into an existing byte[]).
> -
> -    * Fields are separately enumerated (FieldsEnum) from the terms
> -      within each field (TermEnum).  So instead of this:
> +  * Fields are separately enumerated (FieldsEnum) from the terms
> +    within each field (TermEnum).  So instead of this:
>
>         TermEnum termsEnum = ...;
> -       while(termsEnum.next()) {
> -         Term t = termsEnum.term();
> -         System.out.println("field=" + t.field() + "; text=" + t.text());
> +        while(termsEnum.next()) {
> +          Term t = termsEnum.term();
> +          System.out.println("field=" + t.field() + "; text=" + t.text());
>         }
>
> -      Do this:
> -
> +    Do this:
> +
>         FieldsEnum fieldsEnum = ...;
> -       String field;
> -       while((field = fieldsEnum.next()) != null) {
> -         TermsEnum termsEnum = fieldsEnum.terms();
> -         BytesRef text;
> -         while((text = termsEnum.next()) != null) {
> -           System.out.println("field=" + field + "; text=" + text.utf8ToString());
> -         }
> +        String field;
> +        while((field = fieldsEnum.next()) != null) {
> +            TermsEnum termsEnum = fieldsEnum.terms();
> +            BytesRef text;
> +            while((text = termsEnum.next()) != null) {
> +              System.out.println("field=" + field + "; text=" + text.utf8ToString());
> +          }
> +        }
>
> -    * TermDocs is renamed to DocsEnum.  Instead of this:
> +  * TermDocs is renamed to DocsEnum.  Instead of this:
>
>         while(td.next()) {
> -         int doc = td.doc();
> -         ...
> -       }
> +          int doc = td.doc();
> +          ...
> +        }
>
> -      do this:
> +    do this:
>
>         int doc;
> -       while((doc = td.next()) != DocsEnum.NO_MORE_DOCS) {
> -         ...
> -       }
> +        while((doc = td.next()) != DocsEnum.NO_MORE_DOCS) {
> +          ...
> +        }
>
> -      Instead of this:
> -
> +    Instead of this:
> +
>         if (td.skipTo(target)) {
> -         int doc = td.doc();
> -         ...
> -       }
> +          int doc = td.doc();
> +          ...
> +        }
>
> -      do this:
> -
> +    do this:
> +
>         if ((doc=td.advance(target)) != DocsEnum.NO_MORE_DOCS) {
> -         ...
> -       }
> +          ...
> +        }
>
> -      The bulk read API has also changed.  Instead of this:
> +    The bulk read API has also changed.  Instead of this:
>
>         int[] docs = new int[256];
>         int[] freqs = new int[256];
> @@ -145,7 +70,7 @@ LUCENE-1458, LUCENE-2111: Flexible Index
>           // use docs[i], freqs[i]
>         }
>
> -      do this:
> +    do this:
>
>         DocsEnum.BulkReadResult bulk = td.getBulkResult();
>         while(true) {
> @@ -156,319 +81,358 @@ LUCENE-1458, LUCENE-2111: Flexible Index
>           // use bulk.docs.ints[i] and bulk.freqs.ints[i]
>         }
>
> -    * TermPositions is renamed to DocsAndPositionsEnum, and no longer
> -      extends the docs only enumerator (DocsEnum).
> +  * TermPositions is renamed to DocsAndPositionsEnum, and no longer
> +    extends the docs only enumerator (DocsEnum).
>
> -    * Deleted docs are no longer implicitly filtered from
> -      docs/positions enums.  Instead, you pass a Bits
> -      skipDocs (set bits are skipped) when obtaining the enums.  Also,
> -      you can now ask a reader for its deleted docs.
> -
> -    * The docs/positions enums cannot seek to a term.  Instead,
> -      TermsEnum is able to seek, and then you request the
> -      docs/positions enum from that TermsEnum.
> +  * Deleted docs are no longer implicitly filtered from
> +    docs/positions enums.  Instead, you pass a Bits
> +    skipDocs (set bits are skipped) when obtaining the enums.  Also,
> +    you can now ask a reader for its deleted docs.
> +
> +  * The docs/positions enums cannot seek to a term.  Instead,
> +    TermsEnum is able to seek, and then you request the
> +    docs/positions enum from that TermsEnum.
>
> -    * TermsEnum's seek method returns more information.  So instead of
> -      this:
> +  * TermsEnum's seek method returns more information.  So instead of
> +    this:
>
>         Term t;
>         TermEnum termEnum = reader.terms(t);
> -       if (t.equals(termEnum.term())) {
> -         ...
> +        if (t.equals(termEnum.term())) {
> +          ...
>         }
>
> -      do this:
> +    do this:
>
>         TermsEnum termsEnum = ...;
> -       BytesRef text;
> -       if (termsEnum.seek(text) == TermsEnum.SeekStatus.FOUND) {
> -         ...
> -       }
> -
> -      SeekStatus also contains END (enumerator is done) and NOT_FOUND
> -      (term was not found but enumerator is now positioned to the next
> -      term).
> -
> -    * TermsEnum has an ord() method, returning the long numeric
> -      ordinal (ie, first term is 0, next is 1, and so on) for the term
> -      it's not positioned to.  There is also a corresponding seek(long
> -      ord) method.  Note that these methods are optional; in
> -      particular the MultiFields TermsEnum does not implement them.
> -
> -
> -  How you obtain the enums has changed.  The primary entry point is
> -  the Fields class.  If you know your reader is a single segment
> -  reader, do this:
> -
> -    Fields fields = reader.Fields();
> -    if (fields != null) {
> -      ...
> -    }
> +        BytesRef text;
> +        if (termsEnum.seek(text) == TermsEnum.SeekStatus.FOUND) {
> +          ...
> +        }
>
> -  If the reader might be multi-segment, you must do this:
> +    SeekStatus also contains END (enumerator is done) and NOT_FOUND
> +    (term was not found but enumerator is now positioned to the next
> +    term).
> +
> +  * TermsEnum has an ord() method, returning the long numeric
> +    ordinal (ie, first term is 0, next is 1, and so on) for the term
> +    it's not positioned to.  There is also a corresponding seek(long
> +    ord) method.  Note that these methods are optional; in
> +    particular the MultiFields TermsEnum does not implement them.
> +
> +
> +  * How you obtain the enums has changed.  The primary entry point is
> +    the Fields class.  If you know your reader is a single segment
> +    reader, do this:
> +
> +        Fields fields = reader.Fields();
> +        if (fields != null) {
> +          ...
> +        }
> +
> +    If the reader might be multi-segment, you must do this:
>
> -    Fields fields = MultiFields.getFields(reader);
> -    if (fields != null) {
> -      ...
> -    }
> +        Fields fields = MultiFields.getFields(reader);
> +        if (fields != null) {
> +          ...
> +        }
>
> -  The fields may be null (eg if the reader has no fields).
> +    The fields may be null (eg if the reader has no fields).
>
> -  Note that the MultiFields approach entails a performance hit on
> -  MultiReaders, as it must merge terms/docs/positions on the fly. It's
> -  generally better to instead get the sequential readers (use
> -  oal.util.ReaderUtil) and then step through those readers yourself,
> -  if you can (this is how Lucene drives searches).
> +    Note that the MultiFields approach entails a performance hit on
> +    MultiReaders, as it must merge terms/docs/positions on the fly. It's
> +    generally better to instead get the sequential readers (use
> +    oal.util.ReaderUtil) and then step through those readers yourself,
> +    if you can (this is how Lucene drives searches).
> +
> +    If you pass a SegmentReader to MultiFields.fiels it will simply
> +    return reader.fields(), so there is no performance hit in that
> +    case.
> +
> +    Once you have a non-null Fields you can do this:
> +
> +        Terms terms = fields.terms("field");
> +        if (terms != null) {
> +          ...
> +        }
>
> -  If you pass a SegmentReader to MultiFields.fiels it will simply
> -  return reader.fields(), so there is no performance hit in that
> -  case.
> +    The terms may be null (eg if the field does not exist).
>
> -  Once you have a non-null Fields you can do this:
> +    Once you have a non-null terms you can get an enum like this:
>
> -    Terms terms = fields.terms("field");
> -    if (terms != null) {
> -      ...
> -    }
> +        TermsEnum termsEnum = terms.iterator();
>
> -  The terms may be null (eg if the field does not exist).
> +    The returned TermsEnum will not be null.
>
> -  Once you have a non-null terms you can get an enum like this:
> +    You can then .next() through the TermsEnum, or seek.  If you want a
> +    DocsEnum, do this:
>
> -    TermsEnum termsEnum = terms.iterator();
> +        Bits liveDocs = reader.getLiveDocs();
> +        DocsEnum docsEnum = null;
>
> -  The returned TermsEnum will not be null.
> +        docsEnum = termsEnum.docs(liveDocs, docsEnum);
>
> -  You can then .next() through the TermsEnum, or seek.  If you want a
> -  DocsEnum, do this:
> +    You can pass in a prior DocsEnum and it will be reused if possible.
>
> -    Bits liveDocs = reader.getLiveDocs();
> -    DocsEnum docsEnum = null;
> +    Likewise for DocsAndPositionsEnum.
>
> -    docsEnum = termsEnum.docs(liveDocs, docsEnum);
> +    IndexReader has several sugar methods (which just go through the
> +    above steps, under the hood).  Instead of:
>
> -  You can pass in a prior DocsEnum and it will be reused if possible.
> +        Term t;
> +        TermDocs termDocs = reader.termDocs();
> +        termDocs.seek(t);
>
> -  Likewise for DocsAndPositionsEnum.
> +    do this:
>
> -  IndexReader has several sugar methods (which just go through the
> -  above steps, under the hood).  Instead of:
> +        String field;
> +        BytesRef text;
> +        DocsEnum docsEnum = reader.termDocsEnum(reader.getLiveDocs(), field, text);
>
> -    Term t;
> -    TermDocs termDocs = reader.termDocs();
> -    termDocs.seek(t);
> +    Likewise for DocsAndPositionsEnum.
>
> -  do this:
> +## LUCENE-2380: FieldCache.getStrings/Index --> FieldCache.getDocTerms/Index
>
> -    String field;
> -    BytesRef text;
> -    DocsEnum docsEnum = reader.termDocsEnum(reader.getLiveDocs(), field, text);
> +  * The field values returned when sorting by SortField.STRING are now
> +    BytesRef.  You can call value.utf8ToString() to convert back to
> +    string, if necessary.
>
> -  Likewise for DocsAndPositionsEnum.
> +  * In FieldCache, getStrings (returning String[]) has been replaced
> +    with getTerms (returning a FieldCache.DocTerms instance).
> +    DocTerms provides a getTerm method, taking a docID and a BytesRef
> +    to fill (which must not be null), and it fills it in with the
> +    reference to the bytes for that term.
>
> -* LUCENE-2600: remove IndexReader.isDeleted
> +    If you had code like this before:
>
> -  Instead of IndexReader.isDeleted, do this:
> +        String[] values = FieldCache.DEFAULT.getStrings(reader, field);
> +        ...
> +        String aValue = values[docID];
>
> -    import org.apache.lucene.util.Bits;
> -    import org.apache.lucene.index.MultiFields;
> +    you can do this instead:
>
> -    Bits liveDocs = MultiFields.getLiveDocs(indexReader);
> -    if (!liveDocs.get(docID)) {
> -      // document is deleted...
> -    }
> -
> -* LUCENE-2858, LUCENE-3733: The abstract class IndexReader has been
> -  refactored to expose only essential methods to access stored fields
> -  during display of search results. It is no longer possible to retrieve
> -  terms or postings data from the underlying index, not even deletions are
> -  visible anymore. You can still pass IndexReader as constructor parameter
> -  to IndexSearcher and execute your searches; Lucene will automatically
> -  delegate procedures like query rewriting and document collection atomic
> -  subreaders.
> -
> -  If you want to dive deeper into the index and want to write own queries,
> -  take a closer look at the new abstract sub-classes AtomicReader and
> -  CompositeReader:
> -
> -  AtomicReader instances are now the only source of Terms, Postings,
> -  DocValues and FieldCache. Queries are forced to execute on a Atomic
> -  reader on a per-segment basis and FieldCaches are keyed by
> -  AtomicReaders.
> -
> -  Its counterpart CompositeReader exposes a utility method to retrieve
> -  its composites. But watch out, composites are not necessarily atomic.
> -  Next to the added type-safety we also removed the notion of
> -  index-commits and version numbers from the abstract IndexReader, the
> -  associations with IndexWriter were pulled into a specialized
> -  DirectoryReader. To open Directory-based indexes use
> -  DirectoryReader.open(), the corresponding method in IndexReader is now
> -  deprecated for easier migration. Only DirectoryReader supports commits,
> -  versions, and reopening with openIfChanged(). Terms, postings,
> -  docvalues, and norms can from now on only be retrieved using
> -  AtomicReader; DirectoryReader and MultiReader extend CompositeReader,
> -  only offering stored fields and access to the sub-readers (which may be
> -  composite or atomic).
> -
> -  If you have more advanced code dealing with custom Filters, you might
> -  have noticed another new class hierarchy in Lucene (see LUCENE-2831):
> -  IndexReaderContext with corresponding Atomic-/CompositeReaderContext.
> -
> -  The move towards per-segment search Lucene 2.9 exposed lots of custom
> -  Queries and Filters that couldn't handle it. For example, some Filter
> -  implementations expected the IndexReader passed in is identical to the
> -  IndexReader passed to IndexSearcher with all its advantages like
> -  absolute document IDs etc. Obviously this "paradigm-shift" broke lots of
> -  applications and especially those that utilized cross-segment data
> -  structures (like Apache Solr).
> -
> -  In Lucene 4.0, we introduce IndexReaderContexts "searcher-private"
> -  reader hierarchy. During Query or Filter execution Lucene no longer
> -  passes raw readers down Queries, Filters or Collectors; instead
> -  components are provided an AtomicReaderContext (essentially a hierarchy
> -  leaf) holding relative properties like the document-basis in relation to
> -  the top-level reader. This allows Queries & Filter to build up logic
> -  based on document IDs, albeit the per-segment orientation.
> -
> -  There are still valid use-cases where top-level readers ie. "atomic
> -  views" on the index are desirable. Let say you want to iterate all terms
> -  of a complete index for auto-completion or facetting, Lucene provides
> -  utility wrappers like SlowCompositeReaderWrapper (LUCENE-2597) emulating
> -  an AtomicReader. Note: using "atomicity emulators" can cause serious
> -  slowdowns due to the need to merge terms, postings, DocValues, and
> -  FieldCache, use them with care!
> +        DocTerms values = FieldCache.DEFAULT.getTerms(reader, field);
> +        ...
> +        BytesRef term = new BytesRef();
> +        String aValue = values.getTerm(docID, term).utf8ToString();
>
> -* LUCENE-2674: A new idfExplain method was added to Similarity, that
> -  accepts an incoming docFreq.  If you subclass Similarity, make sure
> -  you also override this method on upgrade, otherwise your
> -  customizations won't run for certain MultiTermQuerys.
> +    Note however that it can be costly to convert to String, so it's
> +    better to work directly with the BytesRef.
>
> -* LUCENE-2413: Lucene's core and contrib analyzers, along with Solr's analyzers,
> -  were consolidated into lucene/analysis. During the refactoring some
> -  package names have changed:
> -    - o.a.l.analysis.KeywordAnalyzer -> o.a.l.analysis.core.KeywordAnalyzer
> -    - o.a.l.analysis.KeywordTokenizer -> o.a.l.analysis.core.KeywordTokenizer
> -    - o.a.l.analysis.LetterTokenizer -> o.a.l.analysis.core.LetterTokenizer
> -    - o.a.l.analysis.LowerCaseFilter -> o.a.l.analysis.core.LowerCaseFilter
> -    - o.a.l.analysis.LowerCaseTokenizer -> o.a.l.analysis.core.LowerCaseTokenizer
> -    - o.a.l.analysis.SimpleAnalyzer -> o.a.l.analysis.core.SimpleAnalyzer
> -    - o.a.l.analysis.StopAnalyzer -> o.a.l.analysis.core.StopAnalyzer
> -    - o.a.l.analysis.StopFilter -> o.a.l.analysis.core.StopFilter
> -    - o.a.l.analysis.WhitespaceAnalyzer -> o.a.l.analysis.core.WhitespaceAnalyzer
> -    - o.a.l.analysis.WhitespaceTokenizer -> o.a.l.analysis.core.WhitespaceTokenizer
> -    - o.a.l.analysis.PorterStemFilter -> o.a.l.analysis.en.PorterStemFilter
> -    - o.a.l.analysis.ASCIIFoldingFilter -> o.a.l.analysis.miscellaneous.ASCIIFoldingFilter
> -    - o.a.l.analysis.ISOLatin1AccentFilter -> o.a.l.analysis.miscellaneous.ISOLatin1AccentFilter
> -    - o.a.l.analysis.KeywordMarkerFilter -> o.a.l.analysis.miscellaneous.KeywordMarkerFilter
> -    - o.a.l.analysis.LengthFilter -> o.a.l.analysis.miscellaneous.LengthFilter
> -    - o.a.l.analysis.PerFieldAnalyzerWrapper -> o.a.l.analysis.miscellaneous.PerFieldAnalyzerWrapper
> -    - o.a.l.analysis.TeeSinkTokenFilter -> o.a.l.analysis.sinks.TeeSinkTokenFilter
> -    - o.a.l.analysis.CharFilter -> o.a.l.analysis.charfilter.CharFilter
> -    - o.a.l.analysis.BaseCharFilter -> o.a.l.analysis.charfilter.BaseCharFilter
> -    - o.a.l.analysis.MappingCharFilter -> o.a.l.analysis.charfilter.MappingCharFilter
> -    - o.a.l.analysis.NormalizeCharMap -> o.a.l.analysis.charfilter.NormalizeCharMap
> -    - o.a.l.analysis.CharArraySet -> o.a.l.analysis.util.CharArraySet
> -    - o.a.l.analysis.CharArrayMap -> o.a.l.analysis.util.CharArrayMap
> -    - o.a.l.analysis.ReusableAnalyzerBase -> o.a.l.analysis.util.ReusableAnalyzerBase
> -    - o.a.l.analysis.StopwordAnalyzerBase -> o.a.l.analysis.util.StopwordAnalyzerBase
> -    - o.a.l.analysis.WordListLoader -> o.a.l.analysis.util.WordListLoader
> -    - o.a.l.analysis.CharTokenizer -> o.a.l.analysis.util.CharTokenizer
> -    - o.a.l.util.CharacterUtils -> o.a.l.analysis.util.CharacterUtils
> +  * Similarly, in FieldCache, getStringIndex (returning a StringIndex
> +    instance, with direct arrays int[] order and String[] lookup) has
> +    been replaced with getTermsIndex (returning a
> +    FieldCache.DocTermsIndex instance).  DocTermsIndex provides the
> +    getOrd(int docID) method to lookup the int order for a document,
> +    lookup(int ord, BytesRef reuse) to lookup the term from a given
> +    order, and the sugar method getTerm(int docID, BytesRef reuse)
> +    which internally calls getOrd and then lookup.
>
> -* LUCENE-2514: The option to use a Collator's order (instead of binary order) for
> -  sorting and range queries has been moved to lucene/queries.
> +    If you had code like this before:
>
> -  The Collated TermRangeQuery/Filter has been moved to SlowCollatedTermRangeQuery/Filter,
> -  and the collated sorting has been moved to SlowCollatedStringComparator.
> +        StringIndex idx = FieldCache.DEFAULT.getStringIndex(reader, field);
> +        ...
> +        int ord = idx.order[docID];
> +        String aValue = idx.lookup[ord];
>
> -  Note: this functionality isn't very scalable and if you are using it, consider
> -  indexing collation keys with the collation support in the analysis module instead.
> -
> -  To perform collated range queries, use a suitable collating analyzer: CollationKeyAnalyzer
> -  or ICUCollationKeyAnalyzer, and set qp.setAnalyzeRangeTerms(true).
> -
> -  TermRangeQuery and TermRangeFilter now work purely on bytes. Both have helper factory methods
> -  (newStringRange) similar to the NumericRange API, to easily perform range queries on Strings.
> -
> -* LUCENE-2691: The near-real-time API has moved from IndexWriter to
> -  IndexReader.  Instead of IndexWriter.getReader(), call
> -  IndexReader.open(IndexWriter) or IndexReader.reopen(IndexWriter).
> +    you can do this instead:
>
> -* LUCENE-2690: MultiTermQuery boolean rewrites per segment.
> -  Also MultiTermQuery.getTermsEnum() now takes an AttributeSource. FuzzyTermsEnum
> -  is both consumer and producer of attributes: MTQ.BoostAttribute is
> -  added to the FuzzyTermsEnum and MTQ's rewrite mode consumes it.
> -  The other way round MTQ.TopTermsBooleanQueryRewrite supplys a
> -  global AttributeSource to each segments TermsEnum. The TermsEnum is consumer
> -  and gets the current minimum competitive boosts (MTQ.MaxNonCompetitiveBoostAttribute).
> +        DocTermsIndex idx = FieldCache.DEFAULT.getTermsIndex(reader, field);
> +        ...
> +        int ord = idx.getOrd(docID);
> +        BytesRef term = new BytesRef();
> +        String aValue = idx.lookup(ord, term).utf8ToString();
>
> -* LUCENE-2374: The backwards layer in AttributeImpl was removed. To support correct
> -  reflection of AttributeImpl instances, where the reflection was done using deprecated
> -  toString() parsing, you have to now override reflectWith() to customize output.
> -  toString() is no longer implemented by AttributeImpl, so if you have overridden
> -  toString(), port your customization over to reflectWith(). reflectAsString() would
> -  then return what toString() did before.
> +    Note however that it can be costly to convert to String, so it's
> +    better to work directly with the BytesRef.
>
> -* LUCENE-2236, LUCENE-2912: DefaultSimilarity can no longer be set statically
> -  (and dangerously) for the entire JVM.
> -  Similarity can now be configured on a per-field basis (via PerFieldSimilarityWrapper)
> -  Similarity has a lower-level API, if you want the higher-level vector-space API
> -  like in previous Lucene releases, then look at TFIDFSimilarity.
> +    DocTermsIndex also has a getTermsEnum() method, which returns an
> +    iterator (TermsEnum) over the term values in the index (ie,
> +    iterates ord = 0..numOrd()-1).
>
> -* LUCENE-1076: TieredMergePolicy is now the default merge policy.
> -  It's able to merge non-contiguous segments; this may cause problems
> -  for applications that rely on Lucene's internal document ID
> -  assigment.  If so, you should instead use LogByteSize/DocMergePolicy
> -  during indexing.
> +  * StringComparatorLocale is now more CPU costly than it was before
> +    (it was already very CPU costly since it does not compare using
> +    indexed collation keys; use CollationKeyFilter for better
> +    performance), since it converts BytesRef -> String on the fly.
> +    Also, the field values returned when sorting by SortField.STRING
> +    are now BytesRef.
>
> -* LUCENE-2883: Lucene's o.a.l.search.function ValueSource based functionality, was consolidated
> -  into lucene/queries along with Solr's similar functionality.  The following classes were moved:
> -   - o.a.l.search.function.CustomScoreQuery -> o.a.l.queries.CustomScoreQuery
> -   - o.a.l.search.function.CustomScoreProvider -> o.a.l.queries.CustomScoreProvider
> -   - o.a.l.search.function.NumericIndexDocValueSource -> o.a.l.queries.function.valuesource.NumericIndexDocValueSource
> -  The following lists the replacement classes for those removed:
> -   - o.a.l.search.function.ByteFieldSource -> o.a.l.queries.function.valuesource.ByteFieldSource
> -   - o.a.l.search.function.DocValues -> o.a.l.queries.function.DocValues
> -   - o.a.l.search.function.FieldCacheSource -> o.a.l.queries.function.valuesource.FieldCacheSource
> -   - o.a.l.search.function.FieldScoreQuery ->o.a.l.queries.function.FunctionQuery
> -   - o.a.l.search.function.FloatFieldSource -> o.a.l.queries.function.valuesource.FloatFieldSource
> -   - o.a.l.search.function.IntFieldSource -> o.a.l.queries.function.valuesource.IntFieldSource
> -   - o.a.l.search.function.OrdFieldSource -> o.a.l.queries.function.valuesource.OrdFieldSource
> -   - o.a.l.search.function.ReverseOrdFieldSource -> o.a.l.queries.function.valuesource.ReverseOrdFieldSource
> -   - o.a.l.search.function.ShortFieldSource -> o.a.l.queries.function.valuesource.ShortFieldSource
> -   - o.a.l.search.function.ValueSource -> o.a.l.queries.function.ValueSource
> -   - o.a.l.search.function.ValueSourceQuery -> o.a.l.queries.function.FunctionQuery
> -
> -   DocValues are now named FunctionValues, to not confuse with Lucene's per-document values.
> -
> -* LUCENE-2392: Enable flexible scoring:
> -
> -  The existing "Similarity" api is now TFIDFSimilarity, if you were extending
> -  Similarity before, you should likely extend this instead.
> -
> -  Weight.normalize no longer takes a norm value that incorporates the top-level
> -  boost from outer queries such as BooleanQuery, instead it takes 2 parameters,
> -  the outer boost (topLevelBoost) and the norm. Weight.sumOfSquaredWeights has
> -  been renamed to Weight.getValueForNormalization().
> +  * FieldComparator.StringOrdValComparator has been renamed to
> +    TermOrdValComparator, and now uses BytesRef for its values.
> +    Likewise for StringValComparator, renamed to TermValComparator.
> +    This means when sorting by SortField.STRING or
> +    SortField.STRING_VAL (or directly invoking these comparators) the
> +    values returned in the FieldDoc.fields array will be BytesRef not
> +    String.  You can call the .utf8ToString() method on the BytesRef
> +    instances, if necessary.
>
> -  The scorePayload method now takes a BytesRef. It is never null.
> +## LUCENE-2600: IndexReaders are now read-only
>
> -* LUCENE-3722: Similarity methods and collection/term statistics now take
> -  long instead of int (to enable distributed scoring of > 2B docs).
> -  For example, in TFIDFSimilarity idf(int, int) is now idf(long, long).
> +  Instead of IndexReader.isDeleted, do this:
>
> -* LUCENE-3559: The methods "docFreq" and "maxDoc" on IndexSearcher were removed,
> -  as these are no longer used by the scoring system.
> +      import org.apache.lucene.util.Bits;
> +      import org.apache.lucene.index.MultiFields;
>
> -  If you were using these casually in your code for reasons unrelated to scoring,
> -  call them on the IndexSearcher's reader instead: getIndexReader().
> +      Bits liveDocs = MultiFields.getLiveDocs(indexReader);
> +      if (!liveDocs.get(docID)) {
> +        // document is deleted...
> +      }
> +
> +## LUCENE-2858, LUCENE-3733: IndexReader --> AtomicReader/CompositeReader/DirectoryReader refactoring
>
> -  If you were subclassing IndexSearcher and overriding these methods to alter
> -  scoring, override IndexSearcher's termStatistics() and collectionStatistics()
> -  methods instead.
> +The abstract class IndexReader has been
> +refactored to expose only essential methods to access stored fields
> +during display of search results. It is no longer possible to retrieve
> +terms or postings data from the underlying index, not even deletions are
> +visible anymore. You can still pass IndexReader as constructor parameter
> +to IndexSearcher and execute your searches; Lucene will automatically
> +delegate procedures like query rewriting and document collection atomic
> +subreaders.
> +
> +If you want to dive deeper into the index and want to write own queries,
> +take a closer look at the new abstract sub-classes AtomicReader and
> +CompositeReader:
> +
> +AtomicReader instances are now the only source of Terms, Postings,
> +DocValues and FieldCache. Queries are forced to execute on a Atomic
> +reader on a per-segment basis and FieldCaches are keyed by
> +AtomicReaders.
> +
> +Its counterpart CompositeReader exposes a utility method to retrieve
> +its composites. But watch out, composites are not necessarily atomic.
> +Next to the added type-safety we also removed the notion of
> +index-commits and version numbers from the abstract IndexReader, the
> +associations with IndexWriter were pulled into a specialized
> +DirectoryReader. To open Directory-based indexes use
> +DirectoryReader.open(), the corresponding method in IndexReader is now
> +deprecated for easier migration. Only DirectoryReader supports commits,
> +versions, and reopening with openIfChanged(). Terms, postings,
> +docvalues, and norms can from now on only be retrieved using
> +AtomicReader; DirectoryReader and MultiReader extend CompositeReader,
> +only offering stored fields and access to the sub-readers (which may be
> +composite or atomic).
> +
> +If you have more advanced code dealing with custom Filters, you might
> +have noticed another new class hierarchy in Lucene (see LUCENE-2831):
> +IndexReaderContext with corresponding Atomic-/CompositeReaderContext.
> +
> +The move towards per-segment search Lucene 2.9 exposed lots of custom
> +Queries and Filters that couldn't handle it. For example, some Filter
> +implementations expected the IndexReader passed in is identical to the
> +IndexReader passed to IndexSearcher with all its advantages like
> +absolute document IDs etc. Obviously this "paradigm-shift" broke lots of
> +applications and especially those that utilized cross-segment data
> +structures (like Apache Solr).
> +
> +In Lucene 4.0, we introduce IndexReaderContexts "searcher-private"
> +reader hierarchy. During Query or Filter execution Lucene no longer
> +passes raw readers down Queries, Filters or Collectors; instead
> +components are provided an AtomicReaderContext (essentially a hierarchy
> +leaf) holding relative properties like the document-basis in relation to
> +the top-level reader. This allows Queries & Filter to build up logic
> +based on document IDs, albeit the per-segment orientation.
> +
> +There are still valid use-cases where top-level readers ie. "atomic
> +views" on the index are desirable. Let say you want to iterate all terms
> +of a complete index for auto-completion or facetting, Lucene provides
> +utility wrappers like SlowCompositeReaderWrapper (LUCENE-2597) emulating
> +an AtomicReader. Note: using "atomicity emulators" can cause serious
> +slowdowns due to the need to merge terms, postings, DocValues, and
> +FieldCache, use them with care!
> +
> +## LUCENE-2413: Analyzer package changes
> +
> +Lucene's core and contrib analyzers, along with Solr's analyzers,
> +were consolidated into lucene/analysis. During the refactoring some
> +package names have changed:
> +
> +  - o.a.l.analysis.KeywordAnalyzer -> o.a.l.analysis.core.KeywordAnalyzer
> +  - o.a.l.analysis.KeywordTokenizer -> o.a.l.analysis.core.KeywordTokenizer
> +  - o.a.l.analysis.LetterTokenizer -> o.a.l.analysis.core.LetterTokenizer
> +  - o.a.l.analysis.LowerCaseFilter -> o.a.l.analysis.core.LowerCaseFilter
> +  - o.a.l.analysis.LowerCaseTokenizer -> o.a.l.analysis.core.LowerCaseTokenizer
> +  - o.a.l.analysis.SimpleAnalyzer -> o.a.l.analysis.core.SimpleAnalyzer
> +  - o.a.l.analysis.StopAnalyzer -> o.a.l.analysis.core.StopAnalyzer
> +  - o.a.l.analysis.StopFilter -> o.a.l.analysis.core.StopFilter
> +  - o.a.l.analysis.WhitespaceAnalyzer -> o.a.l.analysis.core.WhitespaceAnalyzer
> +  - o.a.l.analysis.WhitespaceTokenizer -> o.a.l.analysis.core.WhitespaceTokenizer
> +  - o.a.l.analysis.PorterStemFilter -> o.a.l.analysis.en.PorterStemFilter
> +  - o.a.l.analysis.ASCIIFoldingFilter -> o.a.l.analysis.miscellaneous.ASCIIFoldingFilter
> +  - o.a.l.analysis.ISOLatin1AccentFilter -> o.a.l.analysis.miscellaneous.ISOLatin1AccentFilter
> +  - o.a.l.analysis.KeywordMarkerFilter -> o.a.l.analysis.miscellaneous.KeywordMarkerFilter
> +  - o.a.l.analysis.LengthFilter -> o.a.l.analysis.miscellaneous.LengthFilter
> +  - o.a.l.analysis.PerFieldAnalyzerWrapper -> o.a.l.analysis.miscellaneous.PerFieldAnalyzerWrapper
> +  - o.a.l.analysis.TeeSinkTokenFilter -> o.a.l.analysis.sinks.TeeSinkTokenFilter
> +  - o.a.l.analysis.CharFilter -> o.a.l.analysis.charfilter.CharFilter
> +  - o.a.l.analysis.BaseCharFilter -> o.a.l.analysis.charfilter.BaseCharFilter
> +  - o.a.l.analysis.MappingCharFilter -> o.a.l.analysis.charfilter.MappingCharFilter
> +  - o.a.l.analysis.NormalizeCharMap -> o.a.l.analysis.charfilter.NormalizeCharMap
> +  - o.a.l.analysis.CharArraySet -> o.a.l.analysis.util.CharArraySet
> +  - o.a.l.analysis.CharArrayMap -> o.a.l.analysis.util.CharArrayMap
> +  - o.a.l.analysis.ReusableAnalyzerBase -> o.a.l.analysis.util.ReusableAnalyzerBase
> +  - o.a.l.analysis.StopwordAnalyzerBase -> o.a.l.analysis.util.StopwordAnalyzerBase
> +  - o.a.l.analysis.WordListLoader -> o.a.l.analysis.util.WordListLoader
> +  - o.a.l.analysis.CharTokenizer -> o.a.l.analysis.util.CharTokenizer
> +  - o.a.l.util.CharacterUtils -> o.a.l.analysis.util.CharacterUtils
> +
> +## LUCENE-2514: Collators
> +
> +The option to use a Collator's order (instead of binary order) for
> +sorting and range queries has been moved to lucene/queries.
> +The Collated TermRangeQuery/Filter has been moved to SlowCollatedTermRangeQuery/Filter,
> +and the collated sorting has been moved to SlowCollatedStringComparator.
> +
> +Note: this functionality isn't very scalable and if you are using it, consider
> +indexing collation keys with the collation support in the analysis module instead.
> +
> +To perform collated range queries, use a suitable collating analyzer: CollationKeyAnalyzer
> +or ICUCollationKeyAnalyzer, and set qp.setAnalyzeRangeTerms(true).
> +
> +TermRangeQuery and TermRangeFilter now work purely on bytes. Both have helper factory methods
> +(newStringRange) similar to the NumericRange API, to easily perform range queries on Strings.
> +
> +## LUCENE-2883: ValueSource changes
> +
> +Lucene's o.a.l.search.function ValueSource based functionality, was consolidated
> +into lucene/queries along with Solr's similar functionality.  The following classes were moved:
> +
> + - o.a.l.search.function.CustomScoreQuery -> o.a.l.queries.CustomScoreQuery
> + - o.a.l.search.function.CustomScoreProvider -> o.a.l.queries.CustomScoreProvider
> + - o.a.l.search.function.NumericIndexDocValueSource -> o.a.l.queries.function.valuesource.NumericIndexDocValueSource
> +
> +The following lists the replacement classes for those removed:
> +
> + - o.a.l.search.function.ByteFieldSource -> o.a.l.queries.function.valuesource.ByteFieldSource
> + - o.a.l.search.function.DocValues -> o.a.l.queries.function.DocValues
> + - o.a.l.search.function.FieldCacheSource -> o.a.l.queries.function.valuesource.FieldCacheSource
> + - o.a.l.search.function.FieldScoreQuery ->o.a.l.queries.function.FunctionQuery
> + - o.a.l.search.function.FloatFieldSource -> o.a.l.queries.function.valuesource.FloatFieldSource
> + - o.a.l.search.function.IntFieldSource -> o.a.l.queries.function.valuesource.IntFieldSource
> + - o.a.l.search.function.OrdFieldSource -> o.a.l.queries.function.valuesource.OrdFieldSource
> + - o.a.l.search.function.ReverseOrdFieldSource -> o.a.l.queries.function.valuesource.ReverseOrdFieldSource
> + - o.a.l.search.function.ShortFieldSource -> o.a.l.queries.function.valuesource.ShortFieldSource
> + - o.a.l.search.function.ValueSource -> o.a.l.queries.function.ValueSource
> + - o.a.l.search.function.ValueSourceQuery -> o.a.l.queries.function.FunctionQuery
> +
> +DocValues are now named FunctionValues, to not confuse with Lucene's per-document values.
> +
> +## LUCENE-2392: Enable flexible scoring
> +
> +The existing "Similarity" api is now TFIDFSimilarity, if you were extending
> +Similarity before, you should likely extend this instead.
> +
> +Weight.normalize no longer takes a norm value that incorporates the top-level
> +boost from outer queries such as BooleanQuery, instead it takes 2 parameters,
> +the outer boost (topLevelBoost) and the norm. Weight.sumOfSquaredWeights has
> +been renamed to Weight.getValueForNormalization().
> +
> +The scorePayload method now takes a BytesRef. It is never null.
> +
> +## LUCENE-3283: Query parsers moved to separate module
> +
> +Lucene's core o.a.l.queryParser QueryParsers have been consolidated into lucene/queryparser,
> +where other QueryParsers from the codebase will also be placed.  The following classes were moved:
>
> -* LUCENE-3283: Lucene's core o.a.l.queryParser QueryParsers have been consolidated into lucene/queryparser,
> -  where other QueryParsers from the codebase will also be placed.  The following classes were moved:
>   - o.a.l.queryParser.CharStream -> o.a.l.queryparser.classic.CharStream
>   - o.a.l.queryParser.FastCharStream -> o.a.l.queryparser.classic.FastCharStream
>   - o.a.l.queryParser.MultiFieldQueryParser -> o.a.l.queryparser.classic.MultiFieldQueryParser
> @@ -480,9 +444,7 @@ LUCENE-1458, LUCENE-2111: Flexible Index
>   - o.a.l.queryParser.QueryParserToken -> o.a.l.queryparser.classic.Token
>   - o.a.l.queryParser.QueryParserTokenMgrError -> o.a.l.queryparser.classic.TokenMgrError
>
> -
> -
> -* LUCENE-2308,LUCENE-3453: Separate IndexableFieldType from Field instances
> +## LUCENE-2308, LUCENE-3453: Separate IndexableFieldType from Field instances
>
>  With this change, the indexing details (indexed, tokenized, norms,
>  indexOptions, stored, etc.) are moved into a separate FieldType
> @@ -498,15 +460,11 @@ Certain field types are pre-defined sinc
>     not tokenize).  This field turns off norms and indexes only doc
>     IDS (does not index term frequency nor positions).  This field
>     does not store its value, but exposes TYPE_STORED as well.
> -
>   * TextField: indexes and tokenizes a String, Reader or TokenStream
>     value, without term vectors.  This field does not store its value,
>     but exposes TYPE_STORED as well.
> -
>   * StoredField: field that stores its value
> -
>   * DocValuesField: indexes the value as a DocValues field
> -
>   * NumericField: indexes the numeric value so that NumericRangeQuery
>     can be used at search-time.
>
> @@ -515,23 +473,22 @@ instantiate the above class.  If you nee
>  add a separate StoredField to the document, or you can use
>  TYPE_STORED for the field:
>
> -  Field f = new Field("field", "value", StringField.TYPE_STORED);
> +    Field f = new Field("field", "value", StringField.TYPE_STORED);
>
>  Alternatively, if an existing type is close to what you want but you
>  need to make a few changes, you can copy that type and make changes:
>
> -  FieldType bodyType = new FieldType(TextField.TYPE_STORED);
> -  bodyType.setStoreTermVectors(true);
> -
> +    FieldType bodyType = new FieldType(TextField.TYPE_STORED);
> +    bodyType.setStoreTermVectors(true);
>
>  You can of course also create your own FieldType from scratch:
>
> -  FieldType t = new FieldType();
> -  t.setIndexed(true);
> -  t.setStored(true);
> -  t.setOmitNorms(true);
> -  t.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
> -  t.freeze();
> +    FieldType t = new FieldType();
> +    t.setIndexed(true);
> +    t.setStored(true);
> +    t.setOmitNorms(true);
> +    t.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
> +    t.freeze();
>
>  FieldType has a freeze() method to prevent further changes.
>
> @@ -541,65 +498,116 @@ enums.
>
>  When migrating from the 3.x API, if you did this before:
>
> -  new Field("field", "value", Field.Store.NO, Field.Indexed.NOT_ANALYZED_NO_NORMS)
> +    new Field("field", "value", Field.Store.NO, Field.Indexed.NOT_ANALYZED_NO_NORMS)
>
>  you can now do this:
>
> -  new StringField("field", "value")
> +    new StringField("field", "value")
>
>  (though note that StringField indexes DOCS_ONLY).
>
>  If instead the value was stored:
>
> -  new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED_NO_NORMS)
> +    new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED_NO_NORMS)
>
>  you can now do this:
>
> -  new Field("field", "value", StringField.TYPE_STORED)
> +    new Field("field", "value", StringField.TYPE_STORED)
>
>  If you didn't omit norms:
>
> -  new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED)
> +    new Field("field", "value", Field.Store.YES, Field.Indexed.NOT_ANALYZED)
>
>  you can now do this:
>
> -  FieldType ft = new FieldType(StringField.TYPE_STORED);
> -  ft.setOmitNorms(false);
> -  new Field("field", "value", ft)
> +    FieldType ft = new FieldType(StringField.TYPE_STORED);
> +    ft.setOmitNorms(false);
> +    new Field("field", "value", ft)
>
>  If you did this before (value can be String or Reader):
>
> -  new Field("field", value, Field.Store.NO, Field.Indexed.ANALYZED)
> +    new Field("field", value, Field.Store.NO, Field.Indexed.ANALYZED)
>
>  you can now do this:
>
> -  new TextField("field", value)
> +    new TextField("field", value)
>
>  If instead the value was stored:
>
> -  new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED)
> +    new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED)
>
>  you can now do this:
>
> -  new Field("field", value, TextField.TYPE_STORED)
> +    new Field("field", value, TextField.TYPE_STORED)
>
>  If in addition you omit norms:
>
> -  new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED_NO_NORMS)
> +    new Field("field", value, Field.Store.YES, Field.Indexed.ANALYZED_NO_NORMS)
>
>  you can now do this:
>
> -  FieldType ft = new FieldType(TextField.TYPE_STORED);
> -  ft.setOmitNorms(true);
> -  new Field("field", value, ft)
> +    FieldType ft = new FieldType(TextField.TYPE_STORED);
> +    ft.setOmitNorms(true);
> +    new Field("field", value, ft)
>
>  If you did this before (bytes is a byte[]):
>
> -  new Field("field", bytes)
> +    new Field("field", bytes)
>
>  you can now do this:
>
> -  new StoredField("field", bytes)
> +    new StoredField("field", bytes)
> +
> +## Other changes
> +
> +* LUCENE-2674:
> +  A new idfExplain method was added to Similarity, that
> +  accepts an incoming docFreq.  If you subclass Similarity, make sure
> +  you also override this method on upgrade, otherwise your
> +  customizations won't run for certain MultiTermQuerys.
> +
> +* LUCENE-2691: The near-real-time API has moved from IndexWriter to
> +  IndexReader.  Instead of IndexWriter.getReader(), call
> +  IndexReader.open(IndexWriter) or IndexReader.reopen(IndexWriter).
> +
> +* LUCENE-2690: MultiTermQuery boolean rewrites per segment.
> +  Also MultiTermQuery.getTermsEnum() now takes an AttributeSource. FuzzyTermsEnum
> +  is both consumer and producer of attributes: MTQ.BoostAttribute is
> +  added to the FuzzyTermsEnum and MTQ's rewrite mode consumes it.
> +  The other way round MTQ.TopTermsBooleanQueryRewrite supplys a
> +  global AttributeSource to each segments TermsEnum. The TermsEnum is consumer
> +  and gets the current minimum competitive boosts (MTQ.MaxNonCompetitiveBoostAttribute).
> +
> +* LUCENE-2374: The backwards layer in AttributeImpl was removed. To support correct
> +  reflection of AttributeImpl instances, where the reflection was done using deprecated
> +  toString() parsing, you have to now override reflectWith() to customize output.
> +  toString() is no longer implemented by AttributeImpl, so if you have overridden
> +  toString(), port your customization over to reflectWith(). reflectAsString() would
> +  then return what toString() did before.
> +
> +* LUCENE-2236, LUCENE-2912: DefaultSimilarity can no longer be set statically
> +  (and dangerously) for the entire JVM.
> +  Similarity can now be configured on a per-field basis (via PerFieldSimilarityWrapper)
> +  Similarity has a lower-level API, if you want the higher-level vector-space API
> +  like in previous Lucene releases, then look at TFIDFSimilarity.
> +
> +* LUCENE-1076: TieredMergePolicy is now the default merge policy.
> +  It's able to merge non-contiguous segments; this may cause problems
> +  for applications that rely on Lucene's internal document ID
> +  assigment.  If so, you should instead use LogByteSize/DocMergePolicy
> +  during indexing.
> +
> +* LUCENE-3722: Similarity methods and collection/term statistics now take
> +  long instead of int (to enable distributed scoring of > 2B docs).
> +  For example, in TFIDFSimilarity idf(int, int) is now idf(long, long).
> +
> +* LUCENE-3559: The methods "docFreq" and "maxDoc" on IndexSearcher were removed,
> +  as these are no longer used by the scoring system.
> +  If you were using these casually in your code for reasons unrelated to scoring,
> +  call them on the IndexSearcher's reader instead: getIndexReader().
> +  If you were subclassing IndexSearcher and overriding these methods to alter
> +  scoring, override IndexSearcher's termStatistics() and collectionStatistics()
> +  methods instead.
>
>  * LUCENE-3396: Analyzer.tokenStream() and .reusableTokenStream() have been made final.
>   It is now necessary to use Analyzer.TokenStreamComponents to define an analysis process.
> @@ -616,7 +624,6 @@ you can now do this:
>   set integer, float and byte values if a single byte is not sufficient.
>
>  * LUCENE-2621: Term vectors are now accessed via flexible indexing API.
> -
>   If you used IndexReader.getTermFreqVector/s before, you should now
>   use IndexReader.getTermVectors.  The new method returns a Fields
>   instance exposing the inverted index of the one document.  From
>
> Modified: lucene/dev/trunk/lucene/README.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/README.txt?rev=1328978&r1=1328977&r2=1328978&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/README.txt (original)
> +++ lucene/dev/trunk/lucene/README.txt Sun Apr 22 21:15:27 2012
> @@ -1,52 +1,21 @@
> -Apache Lucene README file
> +# Apache Lucene README file
>
> -INTRODUCTION
> +## Introduction
>
>  Lucene is a Java full-text search engine.  Lucene is not a complete
>  application, but rather a code library and API that can easily be used
>  to add search capabilities to applications.
>
> -The Lucene web site is at:
> -  http://lucene.apache.org/
> + * The Lucene web site is at: http://lucene.apache.org/
> + * Please join the Lucene-User mailing list by sending a message to:
> +   java-user-subscribe@lucene.apache.org
>
> -Please join the Lucene-User mailing list by sending a message to:
> -  java-user-subscribe@lucene.apache.org
> -
> -Files in a binary distribution:
> +## Files in a binary distribution
>
>  Files are organized by module, for example in core/:
>
> -core/lucene-core-XX.jar
> +* `core/lucene-core-XX.jar`:
>   The compiled core Lucene library.
>
> -Additional modules contain the same structure:
> -
> -analysis/common/: Analyzers for indexing content in different languages and domains
> -analysis/icu/: Analysis integration with ICU (International Components for Unicode)
> -analysis/kuromoji/: Analyzer for indexing Japanese
> -analysis/morfologik/: Analyzer for indexing Polish
> -analysis/phonetic/: Analyzer for indexing phonetic signatures (for sounds-alike search)
> -analysis/smartcn/: Analyzer for indexing Chinese
> -analysis/stempel/: Analyzer for indexing Polish
> -analysis/uima/: Analysis integration with Apache UIMA
> -benchmark/: System for benchmarking Lucene
> -demo/: Simple example code
> -facet/: Faceted indexing and search capabilities
> -grouping/: Search result grouping
> -highlighter/: Highlights search keywords in results
> -join/: Index-time and Query-time joins for normalized content
> -memory/: Single-document in memory index implementation
> -misc/: Index tools and other miscellaneous code
> -queries/: Filters and Queries that add to core Lucene
> -queryparser/: Query parsers and parsing framework
> -sandbox/: Various third party contributions and new ideas.
> -spatial/: Geospatial search
> -suggest/: Auto-suggest and Spellchecking support
> -test-framework/:  Test Framework for testing Lucene-based applications
> -
> -docs/index.html
> -  The contents of the Lucene website.
> -
> -docs/api/index.html
> -  The Javadoc Lucene API documentation.  This includes the core library,
> -  the test framework, and the demo, as well as all other modules.
> +To review the documentation, read the main documentation page, located at:
> +`docs/index.html`
>
> Modified: lucene/dev/trunk/lucene/build.xml
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/build.xml?rev=1328978&r1=1328977&r2=1328978&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/build.xml (original)
> +++ lucene/dev/trunk/lucene/build.xml Sun Apr 22 21:15:27 2012
> @@ -184,11 +184,11 @@
>   </target>
>
>   <target name="documentation" description="Generate all documentation"
> -    depends="javadocs,changes-to-html,doc-index"/>
> +    depends="javadocs,changes-to-html,process-webpages"/>
>   <target name="javadoc" depends="javadocs"/>
>   <target name="javadocs" description="Generate javadoc" depends="javadocs-lucene-core, javadocs-modules, javadocs-test-framework"/>
>
> -  <target name="doc-index">
> +  <target name="process-webpages" depends="resolve-pegdown">
>     <pathconvert pathsep="|" dirsep="/" property="buildfiles">
>       <fileset dir="." includes="**/build.xml" excludes="build.xml,analysis/*,build/**,tools/**,backwards/**,site/**"/>
>     </pathconvert>
> @@ -205,6 +205,12 @@
>       <param name="buildfiles" expression="${buildfiles}"/>
>       <param name="version" expression="${version}"/>
>     </xslt>
> +
> +    <pegdown todir="${javadoc.dir}">
> +      <fileset dir="." includes="MIGRATE.txt,JRE_VERSION_MIGRATION.txt"/>
> +      <globmapper from="*.txt" to="*.html"/>
> +    </pegdown>
> +
>     <copy todir="${javadoc.dir}">
>       <fileset dir="site/html" includes="**/*"/>
>     </copy>
>
> Modified: lucene/dev/trunk/lucene/common-build.xml
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/common-build.xml?rev=1328978&r1=1328977&r2=1328978&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/common-build.xml (original)
> +++ lucene/dev/trunk/lucene/common-build.xml Sun Apr 22 21:15:27 2012
> @@ -1506,4 +1506,60 @@ ${tests-output}/junit4-*.suites     - pe
>       </scp>
>     </sequential>
>   </macrodef>
> +
> +  <!-- PEGDOWN macro: Before using depend on the target "resolve-pegdown" -->
> +
> +  <target name="resolve-pegdown" unless="pegdown.loaded">
> +    <ivy:cachepath organisation="org.pegdown" module="pegdown" revision="1.1.0"
> +      inline="true" conf="default" type="jar" transitive="true" pathid="pegdown.classpath"/>
> +    <property name="pegdown.loaded" value="true"/>
> +  </target>
> +
> +  <macrodef name="pegdown">
> +    <attribute name="todir"/>
> +    <attribute name="flatten" default="false"/>
> +    <attribute name="overwrite" default="false"/>
> +    <element name="nested" optional="false" implicit="true"/>
> +    <sequential>
> +      <copy todir="@{todir}" flatten="@{flatten}" overwrite="@{overwrite}" verbose="true"
> +        preservelastmodified="false" encoding="UTF-8" outputencoding="UTF-8"
> +      >
> +        <filterchain>
> +          <tokenfilter>
> +            <filetokenizer/>
> +            <replaceregex pattern="\b(LUCENE|SOLR)\-\d+\b" replace="[\0](https://issues.apache.org/jira/browse/\0)" flags="gs"/>
> +            <scriptfilter language="javascript" classpathref="pegdown.classpath"><![CDATA[
> +              importClass(java.lang.StringBuilder);
> +              importClass(org.pegdown.PegDownProcessor);
> +              importClass(org.pegdown.Extensions);
> +              importClass(org.pegdown.FastEncoder);
> +              var markdownSource = self.getToken();
> +              var title = undefined;
> +              if (markdownSource.search(/^(#+\s*)?(.+)[\n\r]/) == 0) {
> +                title = RegExp.$2;
> +                // Convert the first line into a markdown heading, if it is not already:
> +                if (RegExp.$1 == '') {
> +                  markdownSource = '# ' + markdownSource;
> +                }
> +              }
> +              var processor = new PegDownProcessor(
> +                Extensions.ABBREVIATIONS | Extensions.AUTOLINKS |
> +                Extensions.FENCED_CODE_BLOCKS | Extensions.SMARTS
> +              );
> +              var html = new StringBuilder('<html>\n<head>\n');
> +              if (title) {
> +                html.append('<title>').append(FastEncoder.encode(title)).append('</title>\n');
> +              }
> +              html.append('<meta http-equiv="Content-Type" content="text/html; charset=UTF-8">\n')
> +                .append('</head>\n<body>\n')
> +                .append(processor.markdownToHtml(markdownSource))
> +                .append('\n</body>\n</html>\n');
> +              self.setToken(html.toString());
> +            ]]></scriptfilter>
> +          </tokenfilter>
> +        </filterchain>
> +        <nested/>
> +      </copy>
> +    </sequential>
> +  </macrodef>
>  </project>
>
> Modified: lucene/dev/trunk/lucene/site/xsl/index.xsl
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/site/xsl/index.xsl?rev=1328978&r1=1328977&r2=1328978&view=diff
> ==============================================================================
> --- lucene/dev/trunk/lucene/site/xsl/index.xsl (original)
> +++ lucene/dev/trunk/lucene/site/xsl/index.xsl Sun Apr 22 21:15:27 2012
> @@ -37,11 +37,14 @@
>       <body>
>         <div><img src="lucene_green_300.gif"/></div>
>         <h1><xsl:text>Apache Lucene </xsl:text><xsl:value-of select="$version"/><xsl:text> Documentation</xsl:text></h1>
> +        <p>Lucene is a Java full-text search engine. Lucene is not a complete application,
> +        but rather a code library and API that can easily be used to add search capabilities
> +        to applications.</p>
>         <p>
>           This is the official documentation for <b><xsl:text>Apache Lucene </xsl:text>
>           <xsl:value-of select="$version"/></b>. Additional documentation is available in the
>           <a href="http://wiki.apache.org/lucene-java">Wiki</a>.
> -        </p>
> +        </p>
>         <h2>Getting Started</h2>
>         <p>The following section is intended as a "getting started" guide. It has three
>         audiences: first-time users looking to install Apache Lucene in their
> @@ -60,6 +63,8 @@
>         <h2>Reference Documents</h2>
>           <ul>
>             <li><a href="changes/Changes.html">Changes</a>: List of changes in this release.</li>
> +            <li><a href="MIGRATE.html">Migration Guide</a>: What changed in Lucene 4; how to migrate code from Lucene 3.x.</li>
> +            <li><a href="JRE_VERSION_MIGRATION.html">JRE Version Migration</a>: Information about upgrading between major JRE versions.</li>
>             <li><a href="fileformats.html">File Formats</a>: Guide to the index format used by Lucene.</li>
>             <li><a href="core/org/apache/lucene/search/package-summary.html#package_description">Search and Scoring in Lucene</a>: Introduction to how Lucene scores documents.</li>
>             <li><a href="core/org/apache/lucene/search/similarities/TFIDFSimilarity.html">Classic Scoring Formula</a>: Formula of Lucene's classic <a href="http://en.wikipedia.org/wiki/Vector_Space_Model">Vector Space</a> implementation. (look <a href="core/org/apache/lucene/search/similarities/package-summary.html#package_description">here</a> for other models)</li>
>
>



-- 
lucidimagination.com

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org