You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/05/23 18:12:04 UTC

svn commit: r1126565 [1/3] - in /lucene/dev/branches/flexscoring/lucene: ./ backwards/ contrib/ contrib/ant/ contrib/db/ contrib/lucli/ contrib/misc/src/java/org/apache/lucene/index/ contrib/misc/src/java/org/apache/lucene/store/ contrib/misc/src/test/...

Author: rmuir
Date: Mon May 23 16:12:02 2011
New Revision: 1126565

URL: http://svn.apache.org/viewvc?rev=1126565&view=rev
Log:
merge trunk (1103041:1126281)

Added:
    lucene/dev/branches/flexscoring/lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java
    lucene/dev/branches/flexscoring/lucene/contrib/misc/src/test/org/apache/lucene/store/
      - copied from r1126281, lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/store/
    lucene/dev/branches/flexscoring/lucene/contrib/misc/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/contrib/misc/src/test/org/apache/lucene/store/TestNRTCachingDirectory.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterStallControl.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentNorms.java
      - copied, changed from r1126281, lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/CachingCollector.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/CachingCollector.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/BinaryTokenStream.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/BinaryTokenStream.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestBinaryTerms.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBinaryTerms.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/TestCachingCollector.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/src/test/org/apache/lucene/search/TestCachingCollector.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/util/TestVersionComparator.java
      - copied unchanged from r1126281, lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestVersionComparator.java
Removed:
    lucene/dev/branches/flexscoring/lucene/contrib/ant/
    lucene/dev/branches/flexscoring/lucene/contrib/db/
    lucene/dev/branches/flexscoring/lucene/contrib/lucli/
    lucene/dev/branches/flexscoring/lucene/contrib/swing/
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/Healthiness.java
Modified:
    lucene/dev/branches/flexscoring/lucene/   (props changed)
    lucene/dev/branches/flexscoring/lucene/CHANGES.txt
    lucene/dev/branches/flexscoring/lucene/backwards/   (props changed)
    lucene/dev/branches/flexscoring/lucene/build.xml
    lucene/dev/branches/flexscoring/lucene/common-build.xml
    lucene/dev/branches/flexscoring/lucene/contrib/CHANGES.txt
    lucene/dev/branches/flexscoring/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java
    lucene/dev/branches/flexscoring/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MergePolicy.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentReader.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/DocIdSetIterator.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/Directory.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FSDirectory.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMDirectory.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/RAMFile.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/StringHelper.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
    lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
    lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/lucene-contrib/index.xml
    lucene/dev/branches/flexscoring/lucene/src/site/src/documentation/content/xdocs/site.xml
    lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
    lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockAnalyzer.java
    lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/analysis/MockTokenizer.java
    lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/MockRandomMergePolicy.java
    lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/index/codecs/mockrandom/MockRandomCodec.java
    lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
    lucene/dev/branches/flexscoring/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/analysis/TestToken.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFieldsReader.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestGlobalFieldNumbers.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderClone.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexReaderCloneNorms.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestNRTThreads.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/index.32.cfs.zip
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/index/index.32.nocfs.zip
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/TestMultiPhraseQuery.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/search/TestTermScorer.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/util/automaton/TestMinimize.java
    lucene/dev/branches/flexscoring/lucene/src/test/org/apache/lucene/util/automaton/fst/TestFSTs.java

Modified: lucene/dev/branches/flexscoring/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/CHANGES.txt?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/flexscoring/lucene/CHANGES.txt Mon May 23 16:12:02 2011
@@ -162,11 +162,6 @@ Changes in Runtime Behavior
 * LUCENE-2720: IndexWriter throws IndexFormatTooOldException on open, rather 
   than later when e.g. a merge starts. (Shai Erera, Mike McCandless, Uwe Schindler)
 
-* LUCENE-1076: The default merge policy (TieredMergePolicy) is now
-  able to merge non-contiguous segments, which means docIDs no longer
-  necessarily say "in order".  If this is a problem then you can use
-  either of the LogMergePolicy impls.  (Mike McCandless)
-  
 * LUCENE-2881: FieldInfos is now tracked per segment.  Before it was tracked
   per IndexWriter session, which resulted in FieldInfos that had the FieldInfo
   properties from all previous segments combined. Field numbers are now tracked
@@ -280,10 +275,6 @@ API Changes
 * LUCENE-2953: In addition to changes in 3.x, PriorityQueue#initialize(int)
   function was moved into the ctor. (Uwe Schindler, Yonik Seeley)
 
-* LUCENE-3061: IndexWriter's getNextMerge() and merge(OneMerge) are now public
-  (though @lucene.experimental), allowing for custom MergeScheduler 
-  implementations. (Shai Erera)
-
 New features
 
 * LUCENE-2604: Added RegexpQuery support to QueryParser. Regular expressions
@@ -339,9 +330,6 @@ New features
 * LUCENE-2489: Added PerFieldCodecWrapper (in oal.index.codecs) which
   lets you set the Codec per field (Mike McCandless)
 
-* LUCENE-1810: Added FieldSelectorResult.LATENT to not cache lazy loaded fields
-  (Tim Smith, Grant Ingersoll)
-
 * LUCENE-2373: Extend CodecProvider to use SegmentInfosWriter and
   SegmentInfosReader to allow customization of SegmentInfos data.
   (Andrzej Bialecki)
@@ -371,9 +359,6 @@ New features
   use MultiFields static methods directly, instead) if you need to use
   the flex APIs directly on a composite reader.  (Mike McCandless)
 
-* LUCENE-2692: Added several new SpanQuery classes for positional checking
-  (match is in a range, payload is a specific value) (Grant Ingersoll)  
-  
 * LUCENE-2690: MultiTermQuery boolean rewrites per segment.
   (Uwe Schindler, Robert Muir, Mike McCandless, Simon Willnauer)
 
@@ -406,9 +391,6 @@ New features
 * LUCENE-2862: Added TermsEnum.totalTermFreq() and
   Terms.getSumTotalTermFreq().  (Mike McCandless, Robert Muir)
 
-* LUCENE-3001: Added TrieFieldHelper to write solr compatible numeric
-  fields without the solr dependency. (ryan)
-  
 * LUCENE-3003: Added new expert class oal.index.DocTermsOrd,
   refactored from Solr's UnInvertedField, for accessing term ords for
   multi-valued fields, per document.  This is similar to FieldCache in
@@ -445,24 +427,6 @@ Bug fixes
   with more document deletions is requested before a reader with fewer
   deletions, provided they share some segments. (yonik)
 
-* LUCENE-2936: PhraseQuery score explanations were not correctly 
-  identifying matches vs non-matches.  (hossman)
-
-* LUCENE-2996: addIndexes(IndexReader) did not flush before adding the new 
-  indexes, causing existing deletions to be applied on the incoming indexes as 
-  well. (Shai Erera, Mike McCandless)
-
-Test Cases
-
-* LUCENE-3002: added 'tests.iter.min' to control 'tests.iter' by allowing to 
-  stop iterating if at least 'tests.iter.min' ran and a failure occured. 
-  (Shai Erera, Chris Hostetter)
-
-Build
-
-* LUCENE-3006: Building javadocs will fail on warnings by default. 
-  Override with -Dfailonjavadocwarning=false (sarowe, gsingers)
-
 ======================= Lucene 3.x (not yet released) =======================
 
 Changes in backwards compatibility policy
@@ -476,9 +440,15 @@ Changes in backwards compatibility polic
   (Mike McCandless, Shai Erera)
 
 * LUCENE-3084: MergePolicy.OneMerge.segments was changed from
-  SegmentInfos to a List<SegmentInfo>; this is actually a minor change
-  because SegmentInfos itself extends Vector<SegmentInfo>.  (Uwe
-  Schindler, Mike McCandless)
+  SegmentInfos to a List<SegmentInfo>. SegmentInfos itsself was changed
+  to no longer extend Vector<SegmentInfo> (to update code that is using
+  Vector-API, use the new asList() and asSet() methods returning unmodifiable
+  collections; modifying SegmentInfos is now only possible through
+  the explicitely declared methods). IndexWriter.segString() now takes
+  Iterable<SegmentInfo> instead of List<SegmentInfo>. A simple recompile
+  should fix this. MergePolicy and SegmentInfos are internal/experimental
+  APIs not covered by the strict backwards compatibility policy.
+  (Uwe Schindler, Mike McCandless)
 
 Changes in runtime behavior
 
@@ -492,6 +462,13 @@ Changes in runtime behavior
   returns NumericField instances. (Uwe Schindler, Ryan McKinley,
   Mike McCandless)
 
+* LUCENE-1076: Changed the default merge policy from
+  LogByteSizeMergePolicy to TieredMergePolicy, as of Version.LUCENE_32
+  (passed to IndexWriterConfig), which is able to merge non-contiguous
+  segments. This means docIDs no longer necessarily stay "in order"
+  during indexing.  If this is a problem then you can use either of
+  the LogMergePolicy impls.  (Mike McCandless)
+  
 New features
 
 * LUCENE-3082: Added index upgrade tool oal.index.IndexUpgrader
@@ -505,6 +482,10 @@ New features
 * LUCENE-3071: Adding ReversePathHierarchyTokenizer, added skip parameter to 
   PathHierarchyTokenizer (Olivier Favre via ryan)
 
+* LUCENE-1421, LUCENE-3102: added CachingCollector which allow you to cache 
+  document IDs and scores encountered during the search, and "replay" them to 
+  another Collector. (Mike McCandless, Shai Erera)
+
 API Changes
 
 * LUCENE-3061: IndexWriter's getNextMerge() and merge(OneMerge) are now public
@@ -515,6 +496,9 @@ API Changes
   ClassCastException when loading lazy fields or NumericFields.
   (Uwe Schindler, Ryan McKinley, Mike McCandless)
 
+* LUCENE-2027: Directory.touchFile is deprecated and will be removed
+  in 4.0.  (Mike McCandless)
+
 Optimizations
 
 * LUCENE-2990: ArrayUtil/CollectionUtil.*Sort() methods now exit early
@@ -546,6 +530,17 @@ Bug fixes
   PhraseQuery as term with lower doc freq will also have less positions.
   (Uwe Schindler, Robert Muir, Otis Gospodnetic)
 
+* LUCENE-3068: sloppy phrase query failed to match valid documents when multiple 
+  query terms had same position in the query. (Doron Cohen)
+
+* LUCENE-3012: Lucene writes the header now for separate norm files (*.sNNN)
+  (Robert Muir)
+
+Build
+
+* LUCENE-3006: Building javadocs will fail on warnings by default. 
+  Override with -Dfailonjavadocwarning=false (sarowe, gsingers)
+
 Test Cases
 
 * LUCENE-3002: added 'tests.iter.min' to control 'tests.iter' by allowing to 
@@ -1090,6 +1085,12 @@ New features
 
 * LUCENE-2913: Add missing getters to Numeric* classes. (Uwe Schindler)
 
+* LUCENE-1810: Added FieldSelectorResult.LATENT to not cache lazy loaded fields
+  (Tim Smith, Grant Ingersoll)
+
+* LUCENE-2692: Added several new SpanQuery classes for positional checking
+  (match is in a range, payload is a specific value) (Grant Ingersoll)  
+
 Optimizations
 
 * LUCENE-2494: Use CompletionService in ParallelMultiSearcher instead of

Modified: lucene/dev/branches/flexscoring/lucene/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/build.xml?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/build.xml (original)
+++ lucene/dev/branches/flexscoring/lucene/build.xml Mon May 23 16:12:02 2011
@@ -220,19 +220,14 @@
           <!-- make sure the group list below is updated.            -->
           <!-- Also remember to keep site.xml in sync.            -->		  
 
-          <packageset dir="contrib/ant/src/java"/>
-          <packageset dir="contrib/db/bdb-je/src/java"/>
-          <packageset dir="contrib/db/bdb/src/java"/>
           <packageset dir="contrib/demo/src/java"/>
           <packageset dir="contrib/highlighter/src/java"/>
           <packageset dir="contrib/instantiated/src/java"/>
-          <packageset dir="contrib/lucli/src/java"/>
           <packageset dir="contrib/memory/src/java"/>
           <packageset dir="contrib/misc/src/java"/>
           <packageset dir="contrib/queries/src/java"/>
           <packageset dir="contrib/spatial/src/java"/>
           <packageset dir="contrib/spellchecker/src/java"/>
-          <packageset dir="contrib/swing/src/java"/>
           <packageset dir="contrib/wordnet/src/java"/>
           <packageset dir="contrib/xml-query-parser/src/java"/>
           <packageset dir="contrib/queryparser/src/java"/>
@@ -244,20 +239,16 @@
   
           <group title="Core" packages="org.apache.*:org.apache.lucene.analysis:org.apache.lucene.analysis.standard*:org.apache.lucene.analysis.tokenattributes*"/>
     
-          <group title="contrib: Ant" packages="org.apache.lucene.ant*"/>
           <group title="contrib: Demo" packages="org.apache.lucene.demo*"/>
           <group title="contrib: ICU" packages="org.apache.lucene.collation*"/>
-          <group title="contrib: DB" packages="org.apache.lucene.store.db*:org.apache.lucene.store.je*:com.sleepycat*"/>
           <group title="contrib: Highlighter" packages="org.apache.lucene.search.highlight*:org.apache.lucene.search.vectorhighlight*"/>
           <group title="contrib: Instantiated" packages="org.apache.lucene.store.instantiated*"/>
-          <group title="contrib: Lucli" packages="lucli*"/>
           <group title="contrib: Memory" packages="org.apache.lucene.index.memory*"/>
           <group title="contrib: Misc " packages="org.apache.lucene.misc*"/>
           <group title="contrib: Queries" packages="org.apache.lucene.search.similar*:org.apache.lucene.search.regex*:org.apache.regexp*"/>
           <group title="contrib: Query Parser" packages="org.apache.lucene.queryParser.*"/>
           <group title="contrib: Spatial" packages="org.apache.lucene.spatial*"/>
           <group title="contrib: SpellChecker" packages="org.apache.lucene.search.spell*"/>
-          <group title="contrib: Swing" packages="org.apache.lucene.swing*"/>
           <group title="contrib: WordNet" packages="org.apache.lucene.wordnet*"/>
           <group title="contrib: XML Query Parser" packages="org.apache.lucene.xmlparser*"/>
           

Modified: lucene/dev/branches/flexscoring/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/common-build.xml?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/common-build.xml (original)
+++ lucene/dev/branches/flexscoring/lucene/common-build.xml Mon May 23 16:12:02 2011
@@ -849,15 +849,6 @@
       <arg value="-c" />
       <arg value="${basedir}/lib" />
       <arg value="-c" />
-      <arg value="${basedir}/contrib/ant/lib" />
-      <!-- BDB libs are downloaded, don't check them -->
-      <!--<arg value="-c" />
-      <arg value="${toplevel.dir}/lucene/contrib/db/bdb/lib" />
-      <arg value="-c" />
-      <arg value="${toplevel.dir}/lucene/contrib/db/bdb-je/lib" />-->
-      <arg value="-c" />
-      <arg value="${basedir}/contrib/lucli/lib" />
-      <arg value="-c" />
       <arg value="${basedir}/contrib/queries/lib" />
     </java>
   </target>

Modified: lucene/dev/branches/flexscoring/lucene/contrib/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/contrib/CHANGES.txt?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/contrib/CHANGES.txt (original)
+++ lucene/dev/branches/flexscoring/lucene/contrib/CHANGES.txt Mon May 23 16:12:02 2011
@@ -50,6 +50,10 @@ Bug Fixes
 
 ======================= Lucene 3.x (not yet released) =======================
 
+Changes in backwards compatibility policy
+
+ * LUCENE-2981: Removed the following contribs: ant, db, lucli, swing. (Robert Muir)
+
 Changes in runtime behavior
 
  * LUCENE-3086: ItalianAnalyzer now uses ElisionFilter with a set of Italian
@@ -75,10 +79,31 @@ Bug Fixes
    caused a problem if you consumed a tokenstream, then reused it, added different
    attributes to it, and consumed it again.  (Robert Muir, Uwe Schindler)
 
+ * LUCENE-3113: Fixed some minor analysis bugs: double-reset() in ReusableAnalyzerBase
+   and ShingleAnalyzerWrapper, missing end() implementations in PrefixAwareTokenFilter
+   and PrefixAndSuffixAwareTokenFilter, invocations of incrementToken() after it
+   already returned false in CommonGramsQueryFilter, HyphenatedWordsFilter,
+   ShingleFilter, and SynonymsFilter.  (Robert Muir, Steven Rowe, Uwe Schindler)
+
 New Features
 
  * LUCENE-3016: Add analyzer for Latvian.  (Robert Muir)
 
+ * LUCENE-1421: create new grouping contrib module, enabling search
+   results to be grouped by a single-valued indexed field.  This
+   module was factored out of Solr's grouping implementation, but
+   it cannot group by function queries nor arbitrary queries.  (Mike
+   McCandless)
+
+ * LUCENE-3098: add AllGroupsCollector, to collect all unique groups
+   (but in unspecified order).  (Martijn van Groningen via Mike
+   McCandless)
+
+ * LUCENE-3092: Added NRTCachingDirectory in contrib/misc, which
+   caches small segments in RAM.  This is useful, in the near-real-time
+   case where the indexing rate is lowish but the reopen rate is
+   highish, to take load off the IO system.  (Mike McCandless)
+
 Optimizations
 
  * LUCENE-3040: Switch all analysis consumers (highlighter, morelikethis, memory, ...)

Modified: lucene/dev/branches/flexscoring/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java (original)
+++ lucene/dev/branches/flexscoring/lucene/contrib/misc/src/java/org/apache/lucene/index/BalancedSegmentMergePolicy.java Mon May 23 16:12:02 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Set;
 
 /**
@@ -135,7 +136,7 @@ public class BalancedSegmentMergePolicy 
           if (last > 1 || !isOptimized(infos.info(0))) {
 
             spec = new MergeSpecification();
-            spec.add(new OneMerge(infos.range(0, last)));
+            spec.add(new OneMerge(infos.asList().subList(0, last)));
           }
         } else if (last > maxNumSegments) {
 
@@ -192,7 +193,7 @@ public class BalancedSegmentMergePolicy 
       prev = backLink[i][prev];
       int mergeStart = i + prev;
       if((mergeEnd - mergeStart) > 1) {
-        spec.add(new OneMerge(infos.range(mergeStart, mergeEnd)));
+        spec.add(new OneMerge(infos.asList().subList(mergeStart, mergeEnd)));
       } else {
         if(partialExpunge) {
           SegmentInfo info = infos.info(mergeStart);
@@ -208,7 +209,7 @@ public class BalancedSegmentMergePolicy 
     
     if(partialExpunge && maxDelCount > 0) {
       // expunge deletes
-      spec.add(new OneMerge(infos.range(expungeCandidate, expungeCandidate + 1)));
+      spec.add(new OneMerge(Collections.singletonList(infos.info(expungeCandidate))));
     }
     
     return spec;
@@ -250,7 +251,10 @@ public class BalancedSegmentMergePolicy 
     MergeSpecification spec = null;
     
     if(numLargeSegs < numSegs) {
-      SegmentInfos smallSegments = infos.range(numLargeSegs, numSegs);
+      // hack to create a shallow sub-range as SegmentInfos instance,
+      // it does not clone all metadata, but LogMerge does not need it
+      final SegmentInfos smallSegments = new SegmentInfos();
+      smallSegments.rollbackSegmentInfos(infos.asList().subList(numLargeSegs, numSegs));
       spec = super.findMergesToExpungeDeletes(smallSegments);
     }
     
@@ -258,7 +262,7 @@ public class BalancedSegmentMergePolicy 
     for(int i = 0; i < numLargeSegs; i++) {
       SegmentInfo info = infos.info(i);
       if(info.hasDeletions()) {
-        spec.add(new OneMerge(infos.range(i, i + 1)));
+        spec.add(new OneMerge(Collections.singletonList(infos.info(i))));
       }
     }
     return spec;
@@ -296,7 +300,7 @@ public class BalancedSegmentMergePolicy 
       if(totalSmallSegSize < targetSegSize * 2) {
         MergeSpecification spec = findBalancedMerges(infos, numLargeSegs, (numLargeSegs - 1), _partialExpunge);
         if(spec == null) spec = new MergeSpecification(); // should not happen
-        spec.add(new OneMerge(infos.range(numLargeSegs, numSegs)));
+        spec.add(new OneMerge(infos.asList().subList(numLargeSegs, numSegs)));
         return spec;
       } else {
         return findBalancedMerges(infos, numSegs, numLargeSegs, _partialExpunge);
@@ -311,11 +315,13 @@ public class BalancedSegmentMergePolicy 
         if(size(info) < sizeThreshold) break;
         startSeg++;
       }
-      spec.add(new OneMerge(infos.range(startSeg, numSegs)));
+      spec.add(new OneMerge(infos.asList().subList(startSeg, numSegs)));
       return spec;
     } else {
-      // apply the log merge policy to small segments.
-      SegmentInfos smallSegments = infos.range(numLargeSegs, numSegs);
+      // hack to create a shallow sub-range as SegmentInfos instance,
+      // it does not clone all metadata, but LogMerge does not need it
+      final SegmentInfos smallSegments = new SegmentInfos();
+      smallSegments.rollbackSegmentInfos(infos.asList().subList(numLargeSegs, numSegs));
       MergeSpecification spec = super.findMerges(smallSegments);
       
       if(_partialExpunge) {
@@ -342,7 +348,7 @@ public class BalancedSegmentMergePolicy 
       }
     }
     if (maxDelCount > 0) {
-      return new OneMerge(infos.range(expungeCandidate, expungeCandidate + 1));
+      return new OneMerge(Collections.singletonList(infos.info(expungeCandidate)));
     }
     return null;
   }

Modified: lucene/dev/branches/flexscoring/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java (original)
+++ lucene/dev/branches/flexscoring/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java Mon May 23 16:12:02 2011
@@ -631,8 +631,9 @@ public class TestPrecedenceQueryParser e
   }
 
   @Override
-  public void tearDown() {
+  public void tearDown() throws Exception {
     BooleanQuery.setMaxClauseCount(originalMaxClauses);
+    super.tearDown();
   }
 
 }

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/CompoundFileReader.java Mon May 23 16:12:02 2011
@@ -189,12 +189,6 @@ public class CompoundFileReader extends 
         return directory.fileModified(fileName);
     }
 
-    /** Set the modified time of the compound file to now. */
-    @Override
-    public void touchFile(String name) throws IOException {
-        directory.touchFile(fileName);
-    }
-
     /** Not implemented
      * @throws UnsupportedOperationException */
     @Override

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Mon May 23 16:12:02 2011
@@ -741,8 +741,7 @@ class DirectoryReader extends IndexReade
       // case we have to roll back:
       startCommit();
 
-      final SegmentInfos rollbackSegmentInfos = new SegmentInfos();
-      rollbackSegmentInfos.addAll(segmentInfos);
+      final List<SegmentInfo> rollbackSegments = segmentInfos.createBackupSegmentInfos(false);
 
       boolean success = false;
       try {
@@ -774,8 +773,7 @@ class DirectoryReader extends IndexReade
           deleter.refresh();
 
           // Restore all SegmentInfos (in case we pruned some)
-          segmentInfos.clear();
-          segmentInfos.addAll(rollbackSegmentInfos);
+          segmentInfos.rollbackSegmentInfos(rollbackSegments);
         }
       }
 

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Mon May 23 16:12:02 2011
@@ -126,7 +126,6 @@ final class DocumentsWriter {
   final DocumentsWriterPerThreadPool perThreadPool;
   final FlushPolicy flushPolicy;
   final DocumentsWriterFlushControl flushControl;
-  final Healthiness healthiness;
   DocumentsWriter(IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers,
       BufferedDeletesStream bufferedDeletesStream) throws IOException {
     this.directory = directory;
@@ -142,10 +141,7 @@ final class DocumentsWriter {
       flushPolicy = configuredPolicy;
     }
     flushPolicy.init(this);
-    
-    healthiness = new Healthiness();
-    final long maxRamPerDWPT = config.getRAMPerThreadHardLimitMB() * 1024 * 1024;
-    flushControl = new DocumentsWriterFlushControl(this, healthiness, maxRamPerDWPT);
+    flushControl = new DocumentsWriterFlushControl(this, config );
   }
 
   synchronized void deleteQueries(final Query... queries) throws IOException {
@@ -283,31 +279,28 @@ final class DocumentsWriter {
     ensureOpen();
     boolean maybeMerge = false;
     final boolean isUpdate = delTerm != null;
-    if (healthiness.anyStalledThreads()) {
-
-      // Help out flushing any pending DWPTs so we can un-stall:
+    if (flushControl.anyStalledThreads() || flushControl.numQueuedFlushes() > 0) {
+      // Help out flushing any queued DWPTs so we can un-stall:
       if (infoStream != null) {
-        message("WARNING DocumentsWriter has stalled threads; will hijack this thread to flush pending segment(s)");
+        message("DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)");
       }
-
-      // Try pick up pending threads here if possible
-      DocumentsWriterPerThread flushingDWPT;
-      while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
-        // Don't push the delete here since the update could fail!
-        maybeMerge = doFlush(flushingDWPT);
-        if (!healthiness.anyStalledThreads()) {
-          break;
+      do {
+        // Try pick up pending threads here if possible
+        DocumentsWriterPerThread flushingDWPT;
+        while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
+          // Don't push the delete here since the update could fail!
+          maybeMerge |= doFlush(flushingDWPT);
         }
-      }
-
-      if (infoStream != null && healthiness.anyStalledThreads()) {
-        message("WARNING DocumentsWriter still has stalled threads; waiting");
-      }
-
-      healthiness.waitIfStalled(); // block if stalled
+  
+        if (infoStream != null && flushControl.anyStalledThreads()) {
+          message("WARNING DocumentsWriter has stalled threads; waiting");
+        }
+        
+        flushControl.waitIfStalled(); // block if stalled
+      } while (flushControl.numQueuedFlushes() != 0); // still queued DWPTs try help flushing
 
-      if (infoStream != null && healthiness.anyStalledThreads()) {
-        message("WARNING DocumentsWriter done waiting");
+      if (infoStream != null) {
+        message("continue indexing after helpling out flushing DocumentsWriter is healthy");
       }
     }
 
@@ -353,7 +346,6 @@ final class DocumentsWriter {
       maybeMerge = true;
       boolean success = false;
       FlushTicket ticket = null;
-      
       try {
         assert currentFullFlushDelQueue == null
             || flushingDWPT.deleteQueue == currentFullFlushDelQueue : "expected: "
@@ -511,9 +503,7 @@ final class DocumentsWriter {
         anythingFlushed |= doFlush(flushingDWPT);
       }
       // If a concurrent flush is still in flight wait for it
-      while (flushControl.anyFlushing()) {
-        flushControl.waitForFlush();  
-      }
+      flushControl.waitForFlush();  
       if (!anythingFlushed) { // apply deletes if we did not flush any document
         synchronized (ticketQueue) {
           ticketQueue.add(new FlushTicket(flushingDeleteQueue.freezeGlobalBuffer(null), false));

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java Mon May 23 16:12:02 2011
@@ -44,30 +44,32 @@ public final class DocumentsWriterFlushC
   private long activeBytes = 0;
   private long flushBytes = 0;
   private volatile int numPending = 0;
-  private volatile int numFlushing = 0;
   final AtomicBoolean flushDeletes = new AtomicBoolean(false);
   private boolean fullFlush = false;
-  private Queue<DocumentsWriterPerThread> flushQueue = new LinkedList<DocumentsWriterPerThread>();
+  private final Queue<DocumentsWriterPerThread> flushQueue = new LinkedList<DocumentsWriterPerThread>();
   // only for safety reasons if a DWPT is close to the RAM limit
-  private Queue<DocumentsWriterPerThread> blockedFlushes = new LinkedList<DocumentsWriterPerThread>();
-
+  private final Queue<BlockedFlush> blockedFlushes = new LinkedList<BlockedFlush>();
 
+  double maxConfiguredRamBuffer = 0;
   long peakActiveBytes = 0;// only with assert
   long peakFlushBytes = 0;// only with assert
   long peakNetBytes = 0;// only with assert
-  private final Healthiness healthiness;
+  long peakDelta = 0; // only with assert
+  final DocumentsWriterStallControl stallControl;
   private final DocumentsWriterPerThreadPool perThreadPool;
   private final FlushPolicy flushPolicy;
   private boolean closed = false;
   private final HashMap<DocumentsWriterPerThread, Long> flushingWriters = new HashMap<DocumentsWriterPerThread, Long>();
   private final DocumentsWriter documentsWriter;
+  private final IndexWriterConfig config;
 
   DocumentsWriterFlushControl(DocumentsWriter documentsWriter,
-      Healthiness healthiness, long hardMaxBytesPerDWPT) {
-    this.healthiness = healthiness;
+      IndexWriterConfig config) {
+    this.stallControl = new DocumentsWriterStallControl();
     this.perThreadPool = documentsWriter.perThreadPool;
     this.flushPolicy = documentsWriter.flushPolicy;
-    this.hardMaxBytesPerDWPT = hardMaxBytesPerDWPT;
+    this.hardMaxBytesPerDWPT = config.getRAMPerThreadHardLimitMB() * 1024 * 1024;;
+    this.config = config;
     this.documentsWriter = documentsWriter;
   }
 
@@ -82,6 +84,24 @@ public final class DocumentsWriterFlushC
   public synchronized long netBytes() {
     return flushBytes + activeBytes;
   }
+  
+  long stallLimitBytes() {
+    final double maxRamMB = config.getRAMBufferSizeMB();
+    return maxRamMB != IndexWriterConfig.DISABLE_AUTO_FLUSH ? (long)(2 * (maxRamMB * 1024 * 1024)) : Long.MAX_VALUE;
+  }
+  
+  private boolean assertMemory() {
+    final double maxRamMB = config.getRAMBufferSizeMB();
+    if (maxRamMB != IndexWriterConfig.DISABLE_AUTO_FLUSH) {
+      // for this assert we must be tolerant to ram buffer changes!
+      maxConfiguredRamBuffer = Math.max(maxRamMB, maxConfiguredRamBuffer);
+      final long ram = flushBytes + activeBytes;
+      // take peakDelta into account - worst case is that all flushing, pending and blocked DWPT had maxMem and the last doc had the peakDelta 
+      final long expected = (long)(2 * (maxConfiguredRamBuffer * 1024 * 1024)) + ((numPending + numFlushingDWPT() + numBlockedFlushes()) * peakDelta);
+      assert ram <= expected  : "ram was " + ram + " expected: " + expected + " flush mem: " + flushBytes + " active: " + activeBytes ;   
+    }
+    return true;
+  }
 
   private void commitPerThreadBytes(ThreadState perThread) {
     final long delta = perThread.perThread.bytesUsed()
@@ -105,53 +125,62 @@ public final class DocumentsWriterFlushC
     peakActiveBytes = Math.max(peakActiveBytes, activeBytes);
     peakFlushBytes = Math.max(peakFlushBytes, flushBytes);
     peakNetBytes = Math.max(peakNetBytes, netBytes());
+    peakDelta = Math.max(peakDelta, delta);
+    
     return true;
   }
 
   synchronized DocumentsWriterPerThread doAfterDocument(ThreadState perThread,
       boolean isUpdate) {
-    commitPerThreadBytes(perThread);
-    if (!perThread.flushPending) {
-      if (isUpdate) {
-        flushPolicy.onUpdate(this, perThread);
-      } else {
-        flushPolicy.onInsert(this, perThread);
+    try {
+      commitPerThreadBytes(perThread);
+      if (!perThread.flushPending) {
+        if (isUpdate) {
+          flushPolicy.onUpdate(this, perThread);
+        } else {
+          flushPolicy.onInsert(this, perThread);
+        }
+        if (!perThread.flushPending && perThread.bytesUsed > hardMaxBytesPerDWPT) {
+          // Safety check to prevent a single DWPT exceeding its RAM limit. This
+          // is super important since we can not address more than 2048 MB per DWPT
+          setFlushPending(perThread);
+        }
       }
-      if (!perThread.flushPending && perThread.bytesUsed > hardMaxBytesPerDWPT) {
-        // Safety check to prevent a single DWPT exceeding its RAM limit. This
-        // is super important since we can not address more than 2048 MB per DWPT
-        setFlushPending(perThread);
-        if (fullFlush) {
-          DocumentsWriterPerThread toBlock = internalTryCheckOutForFlush(perThread);
-          assert toBlock != null;
-          blockedFlushes.add(toBlock);
+      final DocumentsWriterPerThread flushingDWPT;
+      if (fullFlush) {
+        if (perThread.flushPending) {
+          checkoutAndBlock(perThread);
+          flushingDWPT = nextPendingFlush();
+        } else {
+          flushingDWPT = null;
         }
+      } else {
+       flushingDWPT = tryCheckoutForFlush(perThread);
       }
+      return flushingDWPT;
+    } finally {
+      stallControl.updateStalled(this);
+      assert assertMemory();
     }
-    final DocumentsWriterPerThread flushingDWPT = tryCheckoutForFlush(perThread);
-    healthiness.updateStalled(this);
-    return flushingDWPT;
+    
+    
   }
 
   synchronized void doAfterFlush(DocumentsWriterPerThread dwpt) {
     assert flushingWriters.containsKey(dwpt);
     try {
-      numFlushing--;
       Long bytes = flushingWriters.remove(dwpt);
       flushBytes -= bytes.longValue();
       perThreadPool.recycle(dwpt);
-      healthiness.updateStalled(this);
+      stallControl.updateStalled(this);
+      assert assertMemory();
     } finally {
       notifyAll();
     }
   }
   
-  public synchronized boolean anyFlushing() {
-    return numFlushing != 0;
-  }
-  
   public synchronized void waitForFlush() {
-    if (numFlushing != 0) {
+    while (flushingWriters.size() != 0) {
       try {
         this.wait();
       } catch (InterruptedException e) {
@@ -173,32 +202,51 @@ public final class DocumentsWriterFlushC
       flushBytes += bytes;
       activeBytes -= bytes;
       numPending++; // write access synced
+      assert assertMemory();
     } // don't assert on numDocs since we could hit an abort excp. while selecting that dwpt for flushing
     
   }
 
   synchronized void doOnAbort(ThreadState state) {
-    if (state.flushPending) {
-      flushBytes -= state.bytesUsed;
-    } else {
-      activeBytes -= state.bytesUsed;
+    try {
+      if (state.flushPending) {
+        flushBytes -= state.bytesUsed;
+      } else {
+        activeBytes -= state.bytesUsed;
+      }
+      assert assertMemory();
+      // Take it out of the loop this DWPT is stale
+      perThreadPool.replaceForFlush(state, closed);
+    }finally {
+      stallControl.updateStalled(this);
     }
-    // Take it out of the loop this DWPT is stale
-    perThreadPool.replaceForFlush(state, closed);
-    healthiness.updateStalled(this);
   }
 
   synchronized DocumentsWriterPerThread tryCheckoutForFlush(
       ThreadState perThread) {
-    if (fullFlush) {
-      return null;
+   return perThread.flushPending ? internalTryCheckOutForFlush(perThread) : null;
+  }
+  
+  private void checkoutAndBlock(ThreadState perThread) {
+    perThread.lock();
+    try {
+      assert perThread.flushPending : "can not block non-pending threadstate";
+      assert fullFlush : "can not block if fullFlush == false";
+      final DocumentsWriterPerThread dwpt;
+      final long bytes = perThread.bytesUsed;
+      dwpt = perThreadPool.replaceForFlush(perThread, closed);
+      numPending--;
+      blockedFlushes.add(new BlockedFlush(dwpt, bytes));
+    }finally {
+      perThread.unlock();
     }
-    return internalTryCheckOutForFlush(perThread);
   }
 
   private DocumentsWriterPerThread internalTryCheckOutForFlush(
       ThreadState perThread) {
-    if (perThread.flushPending) {
+    assert Thread.holdsLock(this);
+    assert perThread.flushPending;
+    try {
       // We are pending so all memory is already moved to flushBytes
       if (perThread.tryLock()) {
         try {
@@ -212,15 +260,16 @@ public final class DocumentsWriterFlushC
             // Record the flushing DWPT to reduce flushBytes in doAfterFlush
             flushingWriters.put(dwpt, Long.valueOf(bytes));
             numPending--; // write access synced
-            numFlushing++;
             return dwpt;
           }
         } finally {
           perThread.unlock();
         }
       }
+      return null;
+    } finally {
+      stallControl.updateStalled(this);
     }
-    return null;
   }
 
   @Override
@@ -231,12 +280,13 @@ public final class DocumentsWriterFlushC
 
   DocumentsWriterPerThread nextPendingFlush() {
     synchronized (this) {
-      DocumentsWriterPerThread poll = flushQueue.poll();
-      if (poll != null) {
+      final DocumentsWriterPerThread poll;
+      if ((poll = flushQueue.poll()) != null) {
+        stallControl.updateStalled(this);
         return poll;
-      }  
+      }
     }
-    if (numPending > 0) {
+    if (numPending > 0 && !fullFlush) { // don't check if we are doing a full flush
       final Iterator<ThreadState> allActiveThreads = perThreadPool
           .getActivePerThreadsIterator();
       while (allActiveThreads.hasNext() && numPending > 0) {
@@ -276,8 +326,8 @@ public final class DocumentsWriterFlushC
     return documentsWriter.deleteQueue.numGlobalTermDeletes();
   }
 
-  int numFlushingDWPT() {
-    return numFlushing;
+  synchronized int numFlushingDWPT() {
+    return flushingWriters.size();
   }
   
   public boolean doApplyAllDeletes() {	
@@ -289,7 +339,7 @@ public final class DocumentsWriterFlushC
   }
   
   int numActiveDWPT() {
-    return this.perThreadPool.getMaxThreadStates();
+    return this.perThreadPool.getActiveThreadState();
   }
   
   void markForFullFlush() {
@@ -331,11 +381,11 @@ public final class DocumentsWriterFlushC
             if (!next.flushPending) {
               setFlushPending(next);
             }
+            final DocumentsWriterPerThread flushingDWPT = internalTryCheckOutForFlush(next);
+            assert flushingDWPT != null : "DWPT must never be null here since we hold the lock and it holds documents";
+            assert dwpt == flushingDWPT : "flushControl returned different DWPT";
+            toFlush.add(flushingDWPT);
           }
-          final DocumentsWriterPerThread flushingDWPT = internalTryCheckOutForFlush(next);
-          assert flushingDWPT != null : "DWPT must never be null here since we hold the lock and it holds documents";
-          assert dwpt == flushingDWPT : "flushControl returned different DWPT";
-          toFlush.add(flushingDWPT);
         } else {
           // get the new delete queue from DW
           next.perThread.initialize();
@@ -345,31 +395,54 @@ public final class DocumentsWriterFlushC
       }
     }
     synchronized (this) {
-      assert assertBlockedFlushes(flushingQueue);
-      flushQueue.addAll(blockedFlushes);
-      blockedFlushes.clear();
+      /* make sure we move all DWPT that are where concurrently marked as
+       * pending and moved to blocked are moved over to the flushQueue. There is
+       * a chance that this happens since we marking DWPT for full flush without
+       * blocking indexing.*/
+      pruneBlockedQueue(flushingQueue);   
+      assert assertBlockedFlushes(documentsWriter.deleteQueue);
       flushQueue.addAll(toFlush);
+      stallControl.updateStalled(this);
+    }
+  }
+  
+  /**
+   * Prunes the blockedQueue by removing all DWPT that are associated with the given flush queue. 
+   */
+  private void pruneBlockedQueue(final DocumentsWriterDeleteQueue flushingQueue) {
+    Iterator<BlockedFlush> iterator = blockedFlushes.iterator();
+    while (iterator.hasNext()) {
+      BlockedFlush blockedFlush = iterator.next();
+      if (blockedFlush.dwpt.deleteQueue == flushingQueue) {
+        iterator.remove();
+        assert !flushingWriters.containsKey(blockedFlush.dwpt) : "DWPT is already flushing";
+        // Record the flushing DWPT to reduce flushBytes in doAfterFlush
+        flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
+        // don't decr pending here - its already done when DWPT is blocked
+        flushQueue.add(blockedFlush.dwpt);
+      }
     }
   }
   
   synchronized void finishFullFlush() {
     assert fullFlush;
     assert flushQueue.isEmpty();
+    assert flushingWriters.isEmpty();
     try {
       if (!blockedFlushes.isEmpty()) {
         assert assertBlockedFlushes(documentsWriter.deleteQueue);
-        flushQueue.addAll(blockedFlushes);
-        blockedFlushes.clear();
+        pruneBlockedQueue(documentsWriter.deleteQueue);
+        assert blockedFlushes.isEmpty();
       }
     } finally {
       fullFlush = false;
+      stallControl.updateStalled(this);
     }
   }
   
   boolean assertBlockedFlushes(DocumentsWriterDeleteQueue flushingQueue) {
-    Queue<DocumentsWriterPerThread> flushes = this.blockedFlushes;
-    for (DocumentsWriterPerThread documentsWriterPerThread : flushes) {
-      assert documentsWriterPerThread.deleteQueue == flushingQueue;
+    for (BlockedFlush blockedFlush : blockedFlushes) {
+      assert blockedFlush.dwpt.deleteQueue == flushingQueue;
     }
     return true;
   }
@@ -379,18 +452,65 @@ public final class DocumentsWriterFlushC
       for (DocumentsWriterPerThread dwpt : flushQueue) {
         doAfterFlush(dwpt);
       }
-      for (DocumentsWriterPerThread dwpt : blockedFlushes) {
-        doAfterFlush(dwpt);
+      for (BlockedFlush blockedFlush : blockedFlushes) {
+        flushingWriters.put(blockedFlush.dwpt, Long.valueOf(blockedFlush.bytes));
+        doAfterFlush(blockedFlush.dwpt);
       }
-      
     } finally {
       fullFlush = false;
       flushQueue.clear();
       blockedFlushes.clear();
+      stallControl.updateStalled(this);
     }
   }
   
-  synchronized boolean isFullFlush() {
+  /**
+   * Returns <code>true</code> if a full flush is currently running
+   */
+  synchronized boolean isFullFlush() { // used by assert
     return fullFlush;
   }
+
+  /**
+   * Returns the number of flushes that are already checked out but not yet
+   * actively flushing
+   */
+  synchronized int numQueuedFlushes() {
+    return flushQueue.size();
+  }
+
+  /**
+   * Returns the number of flushes that are checked out but not yet available
+   * for flushing. This only applies during a full flush if a DWPT needs
+   * flushing but must not be flushed until the full flush has finished.
+   */
+  synchronized int numBlockedFlushes() {
+    return blockedFlushes.size();
+  }
+  
+  private static class BlockedFlush {
+    final DocumentsWriterPerThread dwpt;
+    final long bytes;
+    BlockedFlush(DocumentsWriterPerThread dwpt, long bytes) {
+      super();
+      this.dwpt = dwpt;
+      this.bytes = bytes;
+    }
+  }
+
+  /**
+   * This method will block if too many DWPT are currently flushing and no
+   * checked out DWPT are available
+   */
+  void waitIfStalled() {
+      stallControl.waitIfStalled();
+  }
+
+  /**
+   * Returns <code>true</code> iff stalled
+   */
+  boolean anyStalledThreads() {
+    return stallControl.anyStalledThreads();
+  }
+ 
 }
\ No newline at end of file

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Mon May 23 16:12:02 2011
@@ -165,6 +165,13 @@ public abstract class DocumentsWriterPer
   public int getMaxThreadStates() {
     return perThreads.length;
   }
+  
+  /**
+   * Returns the active number of {@link ThreadState} instances.
+   */
+  public int getActiveThreadState() {
+    return numThreadStatesActive;
+  }
 
   /**
    * Returns a new {@link ThreadState} iff any new state is available otherwise

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriter.java Mon May 23 16:12:02 2011
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -221,7 +222,7 @@ public class IndexWriter implements Clos
   private volatile long changeCount; // increments every time a change is completed
   private long lastCommitChangeCount; // last changeCount that was committed
 
-  private SegmentInfos rollbackSegmentInfos;      // segmentInfos we will fallback to if the commit fails
+  private List<SegmentInfo> rollbackSegments;      // list of segmentInfo we will fallback to if the commit fails
 
   volatile SegmentInfos pendingCommit;            // set when a commit is pending (after prepareCommit() & before commit())
   volatile long pendingCommitChangeCount;
@@ -440,14 +441,14 @@ public class IndexWriter implements Clos
     public synchronized boolean infoIsLive(SegmentInfo info) {
       int idx = segmentInfos.indexOf(info);
       assert idx != -1: "info=" + info + " isn't in pool";
-      assert segmentInfos.get(idx) == info: "info=" + info + " doesn't match live info in segmentInfos";
+      assert segmentInfos.info(idx) == info: "info=" + info + " doesn't match live info in segmentInfos";
       return true;
     }
 
     public synchronized SegmentInfo mapToLive(SegmentInfo info) {
       int idx = segmentInfos.indexOf(info);
       if (idx != -1) {
-        info = segmentInfos.get(idx);
+        info = segmentInfos.info(idx);
       }
       return info;
     }
@@ -818,7 +819,7 @@ public class IndexWriter implements Clos
         }
       }
 
-      setRollbackSegmentInfos(segmentInfos);
+      rollbackSegments = segmentInfos.createBackupSegmentInfos(true);
 
       // start with previous field numbers, but new FieldInfos
       globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory);
@@ -862,10 +863,6 @@ public class IndexWriter implements Clos
     }
   }
   
-  private synchronized void setRollbackSegmentInfos(SegmentInfos infos) {
-    rollbackSegmentInfos = (SegmentInfos) infos.clone();
-  }
-
   /**
    * Returns the private {@link IndexWriterConfig}, cloned
    * from the {@link IndexWriterConfig} passed to
@@ -1126,8 +1123,7 @@ public class IndexWriter implements Clos
     else
       count = 0;
 
-    for (int i = 0; i < segmentInfos.size(); i++)
-      count += segmentInfos.info(i).docCount;
+    count += segmentInfos.totalDocCount();
     return count;
   }
 
@@ -1144,8 +1140,7 @@ public class IndexWriter implements Clos
     else
       count = 0;
 
-    for (int i = 0; i < segmentInfos.size(); i++) {
-      final SegmentInfo info = segmentInfos.info(i);
+    for (final SegmentInfo info : segmentInfos) {
       count += info.docCount - numDeletedDocs(info);
     }
     return count;
@@ -1159,9 +1154,11 @@ public class IndexWriter implements Clos
     if (docWriter.anyDeletions()) {
       return true;
     }
-    for (int i = 0; i < segmentInfos.size(); i++)
-      if (segmentInfos.info(i).hasDeletions())
+    for (final SegmentInfo info : segmentInfos) {
+      if (info.hasDeletions()) {
         return true;
+      }
+    }
     return false;
   }
 
@@ -1554,7 +1551,8 @@ public class IndexWriter implements Clos
 
     synchronized(this) {
       resetMergeExceptions();
-      segmentsToOptimize = new HashSet<SegmentInfo>(segmentInfos);
+      segmentsToOptimize.clear();
+      segmentsToOptimize.addAll(segmentInfos.asSet());
       optimizeMaxNumSegments = maxNumSegments;
 
       // Now mark all pending & running merges as optimize
@@ -1778,7 +1776,7 @@ public class IndexWriter implements Clos
 
     final MergePolicy.MergeSpecification spec;
     if (optimize) {
-      spec = mergePolicy.findMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, segmentsToOptimize);
+      spec = mergePolicy.findMergesForOptimize(segmentInfos, maxNumSegmentsOptimize, Collections.unmodifiableSet(segmentsToOptimize));
 
       if (spec != null) {
         final int numMerges = spec.merges.size();
@@ -1889,8 +1887,7 @@ public class IndexWriter implements Clos
         // attempt to commit using this instance of IndexWriter
         // will always write to a new generation ("write
         // once").
-        segmentInfos.clear();
-        segmentInfos.addAll(rollbackSegmentInfos);
+        segmentInfos.rollbackSegmentInfos(rollbackSegments);
 
         docWriter.abort();
 
@@ -2555,7 +2552,7 @@ public class IndexWriter implements Clos
         lastCommitChangeCount = pendingCommitChangeCount;
         segmentInfos.updateGeneration(pendingCommit);
         segmentInfos.setUserData(pendingCommit.getUserData());
-        setRollbackSegmentInfos(pendingCommit);
+        rollbackSegments = pendingCommit.createBackupSegmentInfos(true);
         deleter.checkpoint(pendingCommit, true);
       } finally {
         // Matches the incRef done in startCommit:
@@ -2660,7 +2657,7 @@ public class IndexWriter implements Clos
   final synchronized void applyAllDeletes() throws IOException {
     flushDeletesCount.incrementAndGet();
     final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream
-      .applyDeletes(readerPool, segmentInfos);
+      .applyDeletes(readerPool, segmentInfos.asList());
     if (result.anyDeletes) {
       checkpoint();
     }
@@ -2709,7 +2706,7 @@ public class IndexWriter implements Clos
 
   private void ensureValidMerge(MergePolicy.OneMerge merge) throws IOException {
     for(SegmentInfo info : merge.segments) {
-      if (segmentInfos.indexOf(info) == -1) {
+      if (!segmentInfos.contains(info)) {
         throw new MergePolicy.MergeException("MergePolicy selected a segment (" + info.name + ") that is not in the current index " + segString(), directory);
       }
     }
@@ -2847,39 +2844,13 @@ public class IndexWriter implements Clos
       message("merged segment " + merge.info + " is 100% deleted" +  (keepFullyDeletedSegments ? "" : "; skipping insert"));
     }
 
-    final Set<SegmentInfo> mergedAway = new HashSet<SegmentInfo>(merge.segments);
-    int segIdx = 0;
-    int newSegIdx = 0;
-    boolean inserted = false;
-    final int curSegCount = segmentInfos.size();
-    while(segIdx < curSegCount) {
-      final SegmentInfo info = segmentInfos.info(segIdx++);
-      if (mergedAway.contains(info)) {
-        if (!inserted && (!allDeleted || keepFullyDeletedSegments)) {
-          segmentInfos.set(segIdx-1, merge.info);
-          inserted = true;
-          newSegIdx++;
-        }
-      } else {
-        segmentInfos.set(newSegIdx++, info);
-      }
-    }
-
-    // Either we found place to insert segment, or, we did
-    // not, but only because all segments we merged became
-    // deleted while we are merging, in which case it should
-    // be the case that the new segment is also all deleted:
-    if (!inserted) {
-      assert allDeleted;
-      if (keepFullyDeletedSegments) {
-        segmentInfos.add(0, merge.info);
-      } else {
-        readerPool.drop(merge.info);
-      }
+    final boolean dropSegment = allDeleted && !keepFullyDeletedSegments;
+    segmentInfos.applyMergeChanges(merge, dropSegment);
+    
+    if (dropSegment) {
+      readerPool.drop(merge.info);
     }
-
-    segmentInfos.subList(newSegIdx, segmentInfos.size()).clear();
-
+    
     if (infoStream != null) {
       message("after commit: " + segString());
     }
@@ -3014,7 +2985,7 @@ public class IndexWriter implements Clos
       if (mergingSegments.contains(info)) {
         return false;
       }
-      if (segmentInfos.indexOf(info) == -1) {
+      if (!segmentInfos.contains(info)) {
         return false;
       }
       if (info.dir != directory) {
@@ -3462,7 +3433,7 @@ public class IndexWriter implements Clos
   }
 
   // utility routines for tests
-  SegmentInfo newestSegment() {
+  synchronized SegmentInfo newestSegment() {
     return segmentInfos.size() > 0 ? segmentInfos.info(segmentInfos.size()-1) : null;
   }
 
@@ -3472,19 +3443,18 @@ public class IndexWriter implements Clos
   }
 
   /** @lucene.internal */
-  public synchronized String segString(List<SegmentInfo> infos) throws IOException {
-    StringBuilder buffer = new StringBuilder();
-    final int count = infos.size();
-    for(int i = 0; i < count; i++) {
-      if (i > 0) {
+  public synchronized String segString(Iterable<SegmentInfo> infos) throws IOException {
+    final StringBuilder buffer = new StringBuilder();
+    for(final SegmentInfo s : infos) {
+      if (buffer.length() > 0) {
         buffer.append(' ');
       }
-      buffer.append(segString(infos.get(i)));
+      buffer.append(segString(s));
     }
-
     return buffer.toString();
   }
 
+  /** @lucene.internal */
   public synchronized String segString(SegmentInfo info) throws IOException {
     StringBuilder buffer = new StringBuilder();
     SegmentReader reader = readerPool.getIfExists(info);

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java Mon May 23 16:12:02 2011
@@ -133,10 +133,15 @@ public final class IndexWriterConfig imp
 
   /**
    * Creates a new config that with defaults that match the specified
-   * {@link Version} as well as the default {@link Analyzer}. {@link Version} is
-   * a placeholder for future changes. The default settings are relevant to 3.1
-   * and before. In the future, if different settings will apply to different
-   * versions, they will be documented here.
+   * {@link Version} as well as the default {@link
+   * Analyzer}. If matchVersion is >= {@link
+   * Version#LUCENE_32}, {@link TieredMergePolicy} is used
+   * for merging; else {@link LogByteSizeMergePolicy}.
+   * Note that {@link TieredMergePolicy} is free to select
+   * non-contiguous merges, which means docIDs may not
+   * remain montonic over time.  If this is a problem you
+   * should switch to {@link LogByteSizeMergePolicy} or
+   * {@link LogDocMergePolicy}.
    */
   public IndexWriterConfig(Version matchVersion, Analyzer analyzer) {
     this.matchVersion = matchVersion;
@@ -154,7 +159,11 @@ public final class IndexWriterConfig imp
     indexingChain = DocumentsWriterPerThread.defaultIndexingChain;
     mergedSegmentWarmer = null;
     codecProvider = CodecProvider.getDefault();
-    mergePolicy = new TieredMergePolicy();
+    if (matchVersion.onOrAfter(Version.LUCENE_32)) {
+      mergePolicy = new TieredMergePolicy();
+    } else {
+      mergePolicy = new LogByteSizeMergePolicy();
+    }
     readerPooling = DEFAULT_READER_POOLING;
     indexerThreadPool = new ThreadAffinityDocumentsWriterThreadPool();
     readerTermsIndexDivisor = DEFAULT_READER_TERMS_INDEX_DIVISOR;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java Mon May 23 16:12:02 2011
@@ -242,6 +242,7 @@ public abstract class LogMergePolicy ext
   private MergeSpecification findMergesForOptimizeSizeLimit(
       SegmentInfos infos, int maxNumSegments, int last) throws IOException {
     MergeSpecification spec = new MergeSpecification();
+    final List<SegmentInfo> segments = infos.asList();
 
     int start = last - 1;
     while (start >= 0) {
@@ -254,12 +255,12 @@ public abstract class LogMergePolicy ext
         // unless there is only 1 which is optimized.
         if (last - start - 1 > 1 || (start != last - 1 && !isOptimized(infos.info(start + 1)))) {
           // there is more than 1 segment to the right of this one, or an unoptimized single segment.
-          spec.add(new OneMerge(infos.range(start + 1, last)));
+          spec.add(new OneMerge(segments.subList(start + 1, last)));
         }
         last = start;
       } else if (last - start == mergeFactor) {
         // mergeFactor eligible segments were found, add them as a merge.
-        spec.add(new OneMerge(infos.range(start, last)));
+        spec.add(new OneMerge(segments.subList(start, last)));
         last = start;
       }
       --start;
@@ -267,7 +268,7 @@ public abstract class LogMergePolicy ext
 
     // Add any left-over segments, unless there is just 1 already optimized.
     if (last > 0 && (++start + 1 < last || !isOptimized(infos.info(start)))) {
-      spec.add(new OneMerge(infos.range(start, last)));
+      spec.add(new OneMerge(segments.subList(start, last)));
     }
 
     return spec.merges.size() == 0 ? null : spec;
@@ -280,11 +281,12 @@ public abstract class LogMergePolicy ext
    */
   private MergeSpecification findMergesForOptimizeMaxNumSegments(SegmentInfos infos, int maxNumSegments, int last) throws IOException {
     MergeSpecification spec = new MergeSpecification();
+    final List<SegmentInfo> segments = infos.asList();
     
     // First, enroll all "full" merges (size
     // mergeFactor) to potentially be run concurrently:
     while (last - maxNumSegments + 1 >= mergeFactor) {
-      spec.add(new OneMerge(infos.range(last - mergeFactor, last)));
+      spec.add(new OneMerge(segments.subList(last - mergeFactor, last)));
       last -= mergeFactor;
     }
 
@@ -296,7 +298,7 @@ public abstract class LogMergePolicy ext
         // Since we must optimize down to 1 segment, the
         // choice is simple:
         if (last > 1 || !isOptimized(infos.info(0))) {
-          spec.add(new OneMerge(infos.range(0, last)));
+          spec.add(new OneMerge(segments.subList(0, last)));
         }
       } else if (last > maxNumSegments) {
 
@@ -325,7 +327,7 @@ public abstract class LogMergePolicy ext
           }
         }
 
-        spec.add(new OneMerge(infos.range(bestStart, bestStart + finalMergeSize)));
+        spec.add(new OneMerge(segments.subList(bestStart, bestStart + finalMergeSize)));
       }
     }
     return spec.merges.size() == 0 ? null : spec;
@@ -412,7 +414,8 @@ public abstract class LogMergePolicy ext
   @Override
   public MergeSpecification findMergesToExpungeDeletes(SegmentInfos segmentInfos)
       throws CorruptIndexException, IOException {
-    final int numSegments = segmentInfos.size();
+    final List<SegmentInfo> segments = segmentInfos.asList();
+    final int numSegments = segments.size();
 
     if (verbose())
       message("findMergesToExpungeDeletes: " + numSegments + " segments");
@@ -434,7 +437,7 @@ public abstract class LogMergePolicy ext
           // deletions, so force a merge now:
           if (verbose())
             message("  add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
-          spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, i)));
+          spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i)));
           firstSegmentWithDeletions = i;
         }
       } else if (firstSegmentWithDeletions != -1) {
@@ -443,7 +446,7 @@ public abstract class LogMergePolicy ext
         // mergeFactor segments
         if (verbose())
           message("  add merge " + firstSegmentWithDeletions + " to " + (i-1) + " inclusive");
-        spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, i)));
+        spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, i)));
         firstSegmentWithDeletions = -1;
       }
     }
@@ -451,7 +454,7 @@ public abstract class LogMergePolicy ext
     if (firstSegmentWithDeletions != -1) {
       if (verbose())
         message("  add merge " + firstSegmentWithDeletions + " to " + (numSegments-1) + " inclusive");
-      spec.add(new OneMerge(segmentInfos.range(firstSegmentWithDeletions, numSegments)));
+      spec.add(new OneMerge(segments.subList(firstSegmentWithDeletions, numSegments)));
     }
 
     return spec;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MergePolicy.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/MergePolicy.java Mon May 23 16:12:02 2011
@@ -72,7 +72,7 @@ public abstract class MergePolicy implem
     long mergeGen;                  // used by IndexWriter
     boolean isExternal;             // used by IndexWriter
     int maxNumSegmentsOptimize;     // used by IndexWriter
-    long estimatedMergeBytes;       // used by IndexWriter
+    public long estimatedMergeBytes;       // used by IndexWriter
     List<SegmentReader> readers;        // used by IndexWriter
     List<SegmentReader> readerClones;   // used by IndexWriter
     public final List<SegmentInfo> segments;
@@ -84,7 +84,8 @@ public abstract class MergePolicy implem
     public OneMerge(List<SegmentInfo> segments) {
       if (0 == segments.size())
         throw new RuntimeException("segments must include at least one segment");
-      this.segments = segments;
+      // clone the list, as the in list may be based off original SegmentInfos and may be modified
+      this.segments = new ArrayList<SegmentInfo>(segments);
       int count = 0;
       for(SegmentInfo info : segments) {
         count += info.docCount;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Mon May 23 16:12:02 2011
@@ -40,7 +40,7 @@ import org.apache.lucene.util.Constants;
  *
  * @lucene.experimental
  */
-public final class SegmentInfo {
+public final class SegmentInfo implements Cloneable {
   // TODO: remove with hasVector and hasProx
   private static final int CHECK_FIELDINFO = -2;
   static final int NO = -1;          // e.g. no norms; no deletes;

Modified: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1126565&r1=1126564&r2=1126565&view=diff
==============================================================================
--- lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Mon May 23 16:12:02 2011
@@ -20,13 +20,16 @@ package org.apache.lucene.index;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
-import java.util.Vector;
+import java.util.Set;
 
 import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
 import org.apache.lucene.index.codecs.CodecProvider;
@@ -45,7 +48,7 @@ import org.apache.lucene.util.ThreadInte
  * 
  * @lucene.experimental
  */
-public final class SegmentInfos extends Vector<SegmentInfo> {
+public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
 
   /* 
    * The file format version, a negative number.
@@ -84,7 +87,12 @@ public final class SegmentInfos extends 
   private int format;
   
   private FieldNumberBiMap globalFieldNumberMap; // this segments global field number map - lazy loaded on demand
-
+  
+  private List<SegmentInfo> segments = new ArrayList<SegmentInfo>();
+  private Set<SegmentInfo> segmentSet = new HashSet<SegmentInfo>();
+  private transient List<SegmentInfo> cachedUnmodifiableList;
+  private transient Set<SegmentInfo> cachedUnmodifiableSet;  
+  
   /**
    * If non-null, information about loading segments_N files
    * will be printed here.  @see #setInfoStream.
@@ -107,8 +115,8 @@ public final class SegmentInfos extends 
     return format;
   }
 
-  public final SegmentInfo info(int i) {
-    return get(i);
+  public SegmentInfo info(int i) {
+    return segments.get(i);
   }
 
   /**
@@ -237,7 +245,7 @@ public final class SegmentInfos extends 
     boolean success = false;
 
     // Clear any previous segments:
-    clear();
+    this.clear();
 
     generation = generationFromSegmentsFileName(segmentFileName);
 
@@ -252,7 +260,7 @@ public final class SegmentInfos extends 
       if (!success) {
         // Clear any segment infos we had loaded so we
         // have a clean slate on retry:
-        clear();
+        this.clear();
       }
     }
   }
@@ -349,15 +357,14 @@ public final class SegmentInfos extends 
 
   /** Prunes any segment whose docs are all deleted. */
   public void pruneDeletedSegments() {
-    int segIdx = 0;
-    while(segIdx < size()) {
-      final SegmentInfo info = info(segIdx);
+    for(final Iterator<SegmentInfo> it = segments.iterator(); it.hasNext();) {
+      final SegmentInfo info = it.next();
       if (info.getDelCount() == info.docCount) {
-        remove(segIdx);
-      } else {
-        segIdx++;
+        it.remove();
+        segmentSet.remove(info);
       }
     }
+    assert segmentSet.size() == segments.size();
   }
 
   /**
@@ -367,14 +374,23 @@ public final class SegmentInfos extends 
   
   @Override
   public Object clone() {
-    SegmentInfos sis = (SegmentInfos) super.clone();
-    for(int i=0;i<sis.size();i++) {
-      final SegmentInfo info = sis.info(i);
-      assert info.getSegmentCodecs() != null;
-      sis.set(i, (SegmentInfo) info.clone());
+    try {
+      final SegmentInfos sis = (SegmentInfos) super.clone();
+      // deep clone, first recreate all collections:
+      sis.segments = new ArrayList<SegmentInfo>(size());
+      sis.segmentSet = new HashSet<SegmentInfo>(size());
+      sis.cachedUnmodifiableList = null;
+      sis.cachedUnmodifiableSet = null;
+      for(final SegmentInfo info : this) {
+        assert info.getSegmentCodecs() != null;
+        // dont directly access segments, use add method!!!
+        sis.add((SegmentInfo) info.clone());
+      }
+      sis.userData = new HashMap<String,String>(userData);
+      return sis;
+    } catch (CloneNotSupportedException e) {
+      throw new RuntimeException("should not happen", e);
     }
-    sis.userData = new HashMap<String,String>(userData);
-    return sis;
   }
 
   /**
@@ -742,18 +758,6 @@ public final class SegmentInfos extends 
     protected abstract Object doBody(String segmentFileName) throws CorruptIndexException, IOException;
   }
 
-  /**
-   * Returns a new SegmentInfos containing the SegmentInfo
-   * instances in the specified range first (inclusive) to
-   * last (exclusive), so total number of segments returned
-   * is last-first.
-   */
-  public SegmentInfos range(int first, int last) {
-    SegmentInfos infos = new SegmentInfos(codecs);
-    infos.addAll(super.subList(first, last));
-    return infos;
-  }
-
   // Carry over generation numbers from another SegmentInfos
   void updateGeneration(SegmentInfos other) {
     lastGeneration = other.lastGeneration;
@@ -831,6 +835,10 @@ public final class SegmentInfos extends 
         } catch (Throwable t) {
           // throw orig excp
         }
+      } else {
+        // we must sync here explicitly since during a commit
+        // IW will not sync the global field map. 
+        dir.sync(Collections.singleton(name));
       }
     }
     return version;
@@ -956,7 +964,7 @@ public final class SegmentInfos extends 
   }
   
 
-  public synchronized String toString(Directory directory) {
+  public String toString(Directory directory) {
     StringBuilder buffer = new StringBuilder();
     buffer.append(getCurrentSegmentFileName()).append(": ");
     final int count = size();
@@ -987,8 +995,7 @@ public final class SegmentInfos extends 
    *  remain write once.
    */
   void replace(SegmentInfos other) {
-    clear();
-    addAll(other);
+    rollbackSegmentInfos(other.asList());
     lastGeneration = other.lastGeneration;
     lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
     format = other.format;
@@ -1014,7 +1021,7 @@ public final class SegmentInfos extends 
    * Loads or returns the already loaded the global field number map for this {@link SegmentInfos}.
    * If this {@link SegmentInfos} has no global field number map the returned instance is empty
    */
-  synchronized FieldNumberBiMap getOrLoadGlobalFieldNumberMap(Directory dir) throws IOException {
+  FieldNumberBiMap getOrLoadGlobalFieldNumberMap(Directory dir) throws IOException {
     if (globalFieldNumberMap != null) {
       return globalFieldNumberMap;
     }
@@ -1054,4 +1061,135 @@ public final class SegmentInfos extends 
   long getLastGlobalFieldMapVersion() {
     return lastGlobalFieldMapVersion;
   }
+  
+  /** applies all changes caused by committing a merge to this SegmentInfos */
+  void applyMergeChanges(MergePolicy.OneMerge merge, boolean dropSegment) {
+    final Set<SegmentInfo> mergedAway = new HashSet<SegmentInfo>(merge.segments);
+    boolean inserted = false;
+    int newSegIdx = 0;
+    for (int segIdx = 0, cnt = segments.size(); segIdx < cnt; segIdx++) {
+      assert segIdx >= newSegIdx;
+      final SegmentInfo info = segments.get(segIdx);
+      if (mergedAway.contains(info)) {
+        if (!inserted && !dropSegment) {
+          segments.set(segIdx, merge.info);
+          inserted = true;
+          newSegIdx++;
+        }
+      } else {
+        segments.set(newSegIdx, info);
+        newSegIdx++;
+      }
+    }
+
+    // Either we found place to insert segment, or, we did
+    // not, but only because all segments we merged became
+    // deleted while we are merging, in which case it should
+    // be the case that the new segment is also all deleted,
+    // we insert it at the beginning if it should not be dropped:
+    if (!inserted && !dropSegment) {
+      segments.add(0, merge.info);
+    }
+
+    // the rest of the segments in list are duplicates, so don't remove from map, only list!
+    segments.subList(newSegIdx, segments.size()).clear();
+    
+    // update the Set
+    if (!dropSegment) {
+      segmentSet.add(merge.info);
+    }
+    segmentSet.removeAll(mergedAway);
+    
+    assert segmentSet.size() == segments.size();
+  }
+
+  List<SegmentInfo> createBackupSegmentInfos(boolean cloneChildren) {
+    if (cloneChildren) {
+      final List<SegmentInfo> list = new ArrayList<SegmentInfo>(size());
+      for(final SegmentInfo info : this) {
+        assert info.getSegmentCodecs() != null;
+        list.add((SegmentInfo) info.clone());
+      }
+      return list;
+    } else {
+      return new ArrayList<SegmentInfo>(segments);
+    }
+  }
+  
+  void rollbackSegmentInfos(List<SegmentInfo> infos) {
+    this.clear();
+    this.addAll(infos);
+  }
+  
+  /** Returns an <b>unmodifiable</b> {@link Iterator} of contained segments in order. */
+  // @Override (comment out until Java 6)
+  public Iterator<SegmentInfo> iterator() {
+    return asList().iterator();
+  }
+  
+  /** Returns all contained segments as an <b>unmodifiable</b> {@link List} view. */
+  public List<SegmentInfo> asList() {
+    if (cachedUnmodifiableList == null) {
+      cachedUnmodifiableList = Collections.unmodifiableList(segments);
+    }
+    return cachedUnmodifiableList;
+  }
+  
+  /** Returns all contained segments as an <b>unmodifiable</b> {@link Set} view.
+   * The iterator is not sorted, use {@link List} view or {@link #iterator} to get all segments in order. */
+  public Set<SegmentInfo> asSet() {
+    if (cachedUnmodifiableSet == null) {
+      cachedUnmodifiableSet = Collections.unmodifiableSet(segmentSet);
+    }
+    return cachedUnmodifiableSet;
+  }
+  
+  public int size() {
+    return segments.size();
+  }
+
+  public void add(SegmentInfo si) {
+    if (segmentSet.contains(si)) {
+      throw new IllegalStateException("Cannot add the same segment two times to this SegmentInfos instance");
+    }
+    segments.add(si);
+    segmentSet.add(si);
+    assert segmentSet.size() == segments.size();
+  }
+  
+  public void addAll(Iterable<SegmentInfo> sis) {
+    for (final SegmentInfo si : sis) {
+      this.add(si);
+    }
+  }
+  
+  public void clear() {
+    segments.clear();
+    segmentSet.clear();
+  }
+  
+  public void remove(SegmentInfo si) {
+    final int index = this.indexOf(si);
+    if (index >= 0) {
+      this.remove(index);
+    }
+  }
+  
+  public void remove(int index) {
+    segmentSet.remove(segments.remove(index));
+    assert segmentSet.size() == segments.size();
+  }
+  
+  public boolean contains(SegmentInfo si) {
+    return segmentSet.contains(si);
+  }
+
+  public int indexOf(SegmentInfo si) {
+    if (segmentSet.contains(si)) {
+      return segments.indexOf(si);
+    } else {
+      return -1;
+    }
+  }
+
 }

Copied: lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentNorms.java (from r1126281, lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentNorms.java?p2=lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentNorms.java&p1=lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java&r1=1126281&r2=1126565&rev=1126565&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java (original)
+++ lucene/dev/branches/flexscoring/lucene/src/java/org/apache/lucene/index/SegmentNorms.java Mon May 23 16:12:02 2011
@@ -46,6 +46,7 @@ final class SegmentNorms implements Clon
   private AtomicInteger bytesRef;
   private byte[] bytes;
   private int number;
+  long sum;
 
   boolean dirty;
   boolean rollbackDirty;
@@ -117,6 +118,8 @@ final class SegmentNorms implements Clon
         bytesRef = origNorm.bytesRef;
         bytesRef.incrementAndGet();
 
+        sum = origNorm.sum;
+
         // Once we've loaded the bytes we no longer need
         // origNorm:
         origNorm.decRef();
@@ -135,6 +138,11 @@ final class SegmentNorms implements Clon
         synchronized(in) {
           in.seek(normSeek);
           in.readBytes(bytes, 0, count, false);
+          // nocommit: version the file, and add this sum.
+          sum = 0;
+          for (int i = 0; i < count; i++) {
+            sum += (bytes[i] & 0xff);
+          }
         }
 
         bytesRef = new AtomicInteger(1);