You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2011/01/13 20:53:39 UTC

svn commit: r1058718 [3/18] - in /lucene/dev/branches/realtime_search: ./ lucene/ lucene/contrib/ lucene/contrib/ant/src/java/org/apache/lucene/ant/ lucene/contrib/ant/src/test/org/apache/lucene/ant/ lucene/contrib/benchmark/ lucene/contrib/demo/src/ja...

Modified: lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/TermsFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/TermsFilterTest.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/TermsFilterTest.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/TermsFilterTest.java Thu Jan 13 19:53:21 2011
@@ -21,6 +21,7 @@ import java.util.HashSet;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
@@ -59,23 +60,26 @@ public class TermsFilterTest extends Luc
 			w.addDocument(doc);			
 		}
 		IndexReader reader = new SlowMultiReaderWrapper(w.getReader());
+		assertTrue(reader.getTopReaderContext().isAtomic);
+		AtomicReaderContext context = (AtomicReaderContext) reader.getTopReaderContext();
+		assertTrue(context.isAtomic);
 		w.close();
 		
 		TermsFilter tf=new TermsFilter();
 		tf.addTerm(new Term(fieldName,"19"));
-		OpenBitSet bits = (OpenBitSet)tf.getDocIdSet(reader);
+		OpenBitSet bits = (OpenBitSet)tf.getDocIdSet(context);
 		assertEquals("Must match nothing", 0, bits.cardinality());
 
 		tf.addTerm(new Term(fieldName,"20"));
-		bits = (OpenBitSet)tf.getDocIdSet(reader);
+		bits = (OpenBitSet)tf.getDocIdSet(context);
 		assertEquals("Must match 1", 1, bits.cardinality());
 		
 		tf.addTerm(new Term(fieldName,"10"));
-		bits = (OpenBitSet)tf.getDocIdSet(reader);
+		bits = (OpenBitSet)tf.getDocIdSet(context);
 		assertEquals("Must match 2", 2, bits.cardinality());
 		
 		tf.addTerm(new Term(fieldName,"00"));
-		bits = (OpenBitSet)tf.getDocIdSet(reader);
+		bits = (OpenBitSet)tf.getDocIdSet(context);
 		assertEquals("Must match 2", 2, bits.cardinality());
 		
 		reader.close();

Modified: lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestSpanRegexQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestSpanRegexQuery.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestSpanRegexQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/queries/src/test/org/apache/lucene/search/regex/TestSpanRegexQuery.java Thu Jan 13 19:53:21 2011
@@ -27,10 +27,8 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.MultiSearcher;
 import org.apache.lucene.search.spans.SpanFirstQuery;
 import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
-import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.LockObtainFailedException;
@@ -85,33 +83,6 @@ public class TestSpanRegexQuery extends 
     directory.close();
   }
   
-  public void testSpanRegexBug() throws CorruptIndexException, IOException {
-    createRAMDirectories();
-
-    SpanQuery srq = new SpanMultiTermQueryWrapper<RegexQuery>(new RegexQuery(new Term("field", "a.*")));
-    SpanQuery stq = new SpanMultiTermQueryWrapper<RegexQuery>(new RegexQuery(new Term("field", "b.*")));
-    SpanNearQuery query = new SpanNearQuery(new SpanQuery[] { srq, stq }, 6,
-        true);
-
-    // 1. Search the same store which works
-    IndexSearcher[] arrSearcher = new IndexSearcher[2];
-    arrSearcher[0] = new IndexSearcher(indexStoreA, true);
-    arrSearcher[1] = new IndexSearcher(indexStoreB, true);
-    MultiSearcher searcher = new MultiSearcher(arrSearcher);
-    int numHits = searcher.search(query, null, 1000).totalHits;
-    arrSearcher[0].close();
-    arrSearcher[1].close();
-
-    // Will fail here
-    // We expect 2 but only one matched
-    // The rewriter function only write it once on the first IndexSearcher
-    // So it's using term: a1 b1 to search on the second IndexSearcher
-    // As a result, it won't match the document in the second IndexSearcher
-    assertEquals(2, numHits);
-    indexStoreA.close();
-    indexStoreB.close();
-  }
-  
   private void createRAMDirectories() throws CorruptIndexException,
       LockObtainFailedException, IOException {
     // creating a document to store

Modified: lucene/dev/branches/realtime_search/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/surround/query/BooleanQueryTst.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/surround/query/BooleanQueryTst.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/surround/query/BooleanQueryTst.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/surround/query/BooleanQueryTst.java Thu Jan 13 19:53:21 2011
@@ -19,9 +19,8 @@ package org.apache.lucene.queryParser.su
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Query;
@@ -78,8 +77,8 @@ public class BooleanQueryTst {
     }
 
     @Override
-    public void setNextReader(IndexReader reader, int docBase) throws IOException {
-      this.docBase = docBase;
+    public void setNextReader(AtomicReaderContext context) throws IOException {
+      docBase = context.docBase;
     }
     
     @Override
@@ -122,7 +121,7 @@ public class BooleanQueryTst {
     /* if (verbose) System.out.println("Lucene: " + query.toString()); */
 
     TestCollector tc = new TestCollector();
-    Searcher searcher = new IndexSearcher(dBase.getDb(), true);
+    IndexSearcher searcher = new IndexSearcher(dBase.getDb(), true);
     try {
       searcher.search(query, tc);
     } finally {

Modified: lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/geohash/GeoHashDistanceFilter.java Thu Jan 13 19:53:21 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.spatial.geohas
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.FieldCache.DocTerms;
 import org.apache.lucene.search.Filter;
@@ -62,15 +62,15 @@ public class GeoHashDistanceFilter exten
   }
 
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
 
-    final DocTerms geoHashValues = FieldCache.DEFAULT.getTerms(reader, geoHashField);
+    final DocTerms geoHashValues = FieldCache.DEFAULT.getTerms(context.reader, geoHashField);
     final BytesRef br = new BytesRef();
 
     final int docBase = nextDocBase;
-    nextDocBase += reader.maxDoc();
+    nextDocBase += context.reader.maxDoc();
 
-    return new FilteredDocIdSet(startingFilter.getDocIdSet(reader)) {
+    return new FilteredDocIdSet(startingFilter.getDocIdSet(context)) {
       @Override
       public boolean match(int doc) {
 

Modified: lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/CartesianShapeFilter.java Thu Jan 13 19:53:21 2011
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -45,8 +45,8 @@ public class CartesianShapeFilter extend
   }
   
   @Override
-  public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
-    final Bits delDocs = reader.getDeletedDocs();
+  public DocIdSet getDocIdSet(final AtomicReaderContext context) throws IOException {
+    final Bits delDocs = context.reader.getDeletedDocs();
     final List<Double> area = shape.getArea();
     final int sz = area.size();
     
@@ -58,7 +58,7 @@ public class CartesianShapeFilter extend
       return new DocIdSet() {
         @Override
         public DocIdSetIterator iterator() throws IOException {
-          return reader.termDocsEnum(delDocs, fieldName, bytesRef);
+          return context.reader.termDocsEnum(delDocs, fieldName, bytesRef);
         }
         
         @Override
@@ -67,11 +67,11 @@ public class CartesianShapeFilter extend
         }
       };
     } else {
-      final OpenBitSet bits = new OpenBitSet(reader.maxDoc());
+      final OpenBitSet bits = new OpenBitSet(context.reader.maxDoc());
       for (int i =0; i< sz; i++) {
         double boxId = area.get(i).doubleValue();
         NumericUtils.longToPrefixCoded(NumericUtils.doubleToSortableLong(boxId), 0, bytesRef);
-        final DocsEnum docsEnum = reader.termDocsEnum(delDocs, fieldName, bytesRef);
+        final DocsEnum docsEnum = context.reader.termDocsEnum(delDocs, fieldName, bytesRef);
         if (docsEnum == null) continue;
         // iterate through all documents
         // which have this boxId

Modified: lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFieldComparatorSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFieldComparatorSource.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFieldComparatorSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFieldComparatorSource.java Thu Jan 13 19:53:21 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.spatial.tier;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.FieldComparator;
 import org.apache.lucene.search.FieldComparatorSource;
@@ -108,16 +108,14 @@ public class DistanceFieldComparatorSour
 
 		}
 
-		@Override
-                public FieldComparator setNextReader(IndexReader reader, int docBase)
-                  throws IOException {
-			
-			// each reader in a segmented base
-			// has an offset based on the maxDocs of previous readers
-			offset = docBase;
-
-                        return this;
-		}
+    @Override
+    public FieldComparator setNextReader(AtomicReaderContext context)
+        throws IOException {
+      // each reader in a segmented base
+      // has an offset based on the maxDocs of previous readers
+      offset = context.docBase;
+      return this;
+    }
 
 		@Override
 		public Comparable<Double> value(int slot) {

Modified: lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/java/org/apache/lucene/spatial/tier/LatLongDistanceFilter.java Thu Jan 13 19:53:21 2011
@@ -18,7 +18,8 @@
 package org.apache.lucene.spatial.tier;
 
 import java.io.IOException;
-import org.apache.lucene.index.IndexReader;
+
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FilteredDocIdSet;
 import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Filter;
@@ -64,15 +65,15 @@ public class LatLongDistanceFilter exten
   }
   
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
 
-    final double[] latIndex = FieldCache.DEFAULT.getDoubles(reader, latField);
-    final double[] lngIndex = FieldCache.DEFAULT.getDoubles(reader, lngField);
+    final double[] latIndex = FieldCache.DEFAULT.getDoubles(context.reader, latField);
+    final double[] lngIndex = FieldCache.DEFAULT.getDoubles(context.reader, lngField);
 
     final int docBase = nextDocBase;
-    nextDocBase += reader.maxDoc();
+    nextDocBase += context.reader.maxDoc();
 
-    return new FilteredDocIdSet(startingFilter.getDocIdSet(reader)) {
+    return new FilteredDocIdSet(startingFilter.getDocIdSet(context)) {
       @Override
       protected boolean match(int doc) {
         double x = latIndex[doc];

Modified: lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java Thu Jan 13 19:53:21 2011
@@ -252,7 +252,7 @@ public class TestCartesian extends Lucen
 
     // Perform the search, using the term query, the serial chain filter, and the
     // distance sort
-    TopDocs hits = searcher.search(customScore.createWeight(searcher),null, 1000, sort);
+    TopDocs hits = searcher.search(customScore,null, 1000, sort);
     int results = hits.totalHits;
     ScoreDoc[] scoreDocs = hits.scoreDocs; 
     
@@ -348,7 +348,7 @@ public class TestCartesian extends Lucen
 
     // Perform the search, using the term query, the serial chain filter, and the
     // distance sort
-    TopDocs hits = searcher.search(customScore.createWeight(searcher),null, 1000, sort);
+    TopDocs hits = searcher.search(customScore,null, 1000, sort);
     int results = hits.totalHits;
     ScoreDoc[] scoreDocs = hits.scoreDocs; 
 
@@ -444,7 +444,7 @@ public class TestCartesian extends Lucen
     
       // Perform the search, using the term query, the serial chain filter, and the
       // distance sort
-      TopDocs hits = searcher.search(customScore.createWeight(searcher),null, 1000, sort);
+      TopDocs hits = searcher.search(customScore,null, 1000, sort);
       int results = hits.totalHits;
       ScoreDoc[] scoreDocs = hits.scoreDocs; 
     
@@ -539,7 +539,7 @@ public class TestCartesian extends Lucen
 	    
       // Perform the search, using the term query, the serial chain filter, and the
       // distance sort
-      TopDocs hits = searcher.search(customScore.createWeight(searcher),dq.getFilter(), 1000); //,sort);
+      TopDocs hits = searcher.search(customScore,dq.getFilter(), 1000); //,sort);
       int results = hits.totalHits;
       ScoreDoc[] scoreDocs = hits.scoreDocs; 
 	    

Modified: lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java Thu Jan 13 19:53:21 2011
@@ -22,14 +22,17 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.NumericField;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.store.Directory;
 
+
 public class TestDistance extends LuceneTestCase {
   
   private Directory directory;
@@ -100,9 +103,9 @@ public class TestDistance extends Lucene
     LatLongDistanceFilter f = new LatLongDistanceFilter(new QueryWrapperFilter(new MatchAllDocsQuery()),
                                                         lat, lng, 1.0, latField, lngField);
 
-    IndexReader[] readers = r.getSequentialSubReaders();
-    for(int i=0;i<readers.length;i++) {
-      f.getDocIdSet(readers[i]);
+    AtomicReaderContext[] leaves = ReaderUtil.leaves(r.getTopReaderContext());
+    for (int i = 0; i < leaves.length; i++) {
+      f.getDocIdSet(leaves[i]);
     }
     r.close();
   }

Modified: lucene/dev/branches/realtime_search/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/spellchecker/src/java/org/apache/lucene/search/spell/SpellChecker.java Thu Jan 13 19:53:21 2011
@@ -18,8 +18,10 @@ package org.apache.lucene.search.spell;
  */
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.Iterator;
+import java.util.List;
 
 import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
 import org.apache.lucene.document.Document;
@@ -30,6 +32,8 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
@@ -38,7 +42,10 @@ import org.apache.lucene.search.ScoreDoc
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.Version;
+import org.apache.lucene.util.VirtualMethod;
 
 /**
  * <p>
@@ -492,35 +499,64 @@ public class SpellChecker implements jav
    * @param dict Dictionary to index
    * @param mergeFactor mergeFactor to use when indexing
    * @param ramMB the max amount or memory in MB to use
+   * @param optimize whether or not the spellcheck index should be optimized
    * @throws AlreadyClosedException if the Spellchecker is already closed
    * @throws IOException
    */
-  public void indexDictionary(Dictionary dict, int mergeFactor, int ramMB) throws IOException {
+  public final void indexDictionary(Dictionary dict, int mergeFactor, int ramMB, boolean optimize) throws IOException {
     synchronized (modifyCurrentIndexLock) {
       ensureOpen();
       final Directory dir = this.spellIndex;
       final IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(Version.LUCENE_CURRENT, new WhitespaceAnalyzer(Version.LUCENE_CURRENT)).setRAMBufferSizeMB(ramMB));
       ((LogMergePolicy) writer.getConfig().getMergePolicy()).setMergeFactor(mergeFactor);
+      IndexSearcher indexSearcher = obtainSearcher();
+      final List<TermsEnum> termsEnums = new ArrayList<TermsEnum>();
+
+      if (searcher.maxDoc() > 0) {
+        new ReaderUtil.Gather(searcher.getIndexReader()) {
+          @Override
+          protected void add(int base, IndexReader r) throws IOException {
+            Terms terms = r.terms(F_WORD);
+            if (terms != null)
+              termsEnums.add(terms.iterator());
+          }
+        }.run();
+      }
+      
+      boolean isEmpty = termsEnums.isEmpty();
+
+      try { 
+        Iterator<String> iter = dict.getWordsIterator();
+        BytesRef currentTerm = new BytesRef();
+        
+        terms: while (iter.hasNext()) {
+          String word = iter.next();
+  
+          int len = word.length();
+          if (len < 3) {
+            continue; // too short we bail but "too long" is fine...
+          }
   
-      Iterator<String> iter = dict.getWordsIterator();
-      while (iter.hasNext()) {
-        String word = iter.next();
-  
-        int len = word.length();
-        if (len < 3) {
-          continue; // too short we bail but "too long" is fine...
-        }
+          if (!isEmpty) {
+            // we have a non-empty index, check if the term exists
+            currentTerm.copy(word);
+            for (TermsEnum te : termsEnums) {
+              if (te.seek(currentTerm, false) == TermsEnum.SeekStatus.FOUND) {
+                continue terms;
+              }
+            }
+          }
   
-        if (this.exist(word)) { // if the word already exist in the gramindex
-          continue;
+          // ok index the word
+          Document doc = createDocument(word, getMin(len), getMax(len));
+          writer.addDocument(doc);
         }
-  
-        // ok index the word
-        Document doc = createDocument(word, getMin(len), getMax(len));
-        writer.addDocument(doc);
+      } finally {
+        releaseSearcher(indexSearcher);
       }
       // close writer
-      writer.optimize();
+      if (optimize)
+        writer.optimize();
       writer.close();
       // also re-open the spell index to see our own changes when the next suggestion
       // is fetched:
@@ -531,10 +567,21 @@ public class SpellChecker implements jav
   /**
    * Indexes the data from the given {@link Dictionary}.
    * @param dict the dictionary to index
+   * @param mergeFactor mergeFactor to use when indexing
+   * @param ramMB the max amount or memory in MB to use
+   * @throws IOException
+   */
+  public final void indexDictionary(Dictionary dict, int mergeFactor, int ramMB) throws IOException {
+    indexDictionary(dict, mergeFactor, ramMB, true);
+  }
+  
+  /**
+   * Indexes the data from the given {@link Dictionary}.
+   * @param dict the dictionary to index
    * @throws IOException
    */
-  public void indexDictionary(Dictionary dict) throws IOException {
-    indexDictionary(dict, 300, 10);
+  public final void indexDictionary(Dictionary dict) throws IOException {
+    indexDictionary(dict, 300, (int)IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB);
   }
 
   private static int getMin(int l) {
@@ -559,7 +606,12 @@ public class SpellChecker implements jav
 
   private static Document createDocument(String text, int ng1, int ng2) {
     Document doc = new Document();
-    doc.add(new Field(F_WORD, text, Field.Store.YES, Field.Index.NOT_ANALYZED)); // orig term
+    // the word field is never queried on... its indexed so it can be quickly
+    // checked for rebuild (and stored for retrieval). Doesn't need norms or TF/pos
+    Field f = new Field(F_WORD, text, Field.Store.YES, Field.Index.NOT_ANALYZED);
+    f.setOmitTermFreqAndPositions(true);
+    f.setOmitNorms(true);
+    doc.add(f); // orig term
     addGram(text, doc, ng1, ng2);
     return doc;
   }
@@ -573,12 +625,20 @@ public class SpellChecker implements jav
         String gram = text.substring(i, i + ng);
         doc.add(new Field(key, gram, Field.Store.NO, Field.Index.NOT_ANALYZED));
         if (i == 0) {
-          doc.add(new Field("start" + ng, gram, Field.Store.NO, Field.Index.NOT_ANALYZED));
+          // only one term possible in the startXXField, TF/pos and norms aren't needed.
+          Field startField = new Field("start" + ng, gram, Field.Store.NO, Field.Index.NOT_ANALYZED);
+          startField.setOmitTermFreqAndPositions(true);
+          startField.setOmitNorms(true);
+          doc.add(startField);
         }
         end = gram;
       }
       if (end != null) { // may not be present if len==ng1
-        doc.add(new Field("end" + ng, end, Field.Store.NO, Field.Index.NOT_ANALYZED));
+        // only one term possible in the endXXField, TF/pos and norms aren't needed.
+        Field endField = new Field("end" + ng, end, Field.Store.NO, Field.Index.NOT_ANALYZED);
+        endField.setOmitTermFreqAndPositions(true);
+        endField.setOmitNorms(true);
+        doc.add(endField);
       }
     }
   }

Modified: lucene/dev/branches/realtime_search/lucene/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/swing/src/java/org/apache/lucene/swing/models/ListSearcher.java Thu Jan 13 19:53:21 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.document.Fielda
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.queryParser.MultiFieldQueryParser;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.IndexSearcher;
@@ -192,7 +193,7 @@ public class ListSearcher extends Abstra
       }
 
       @Override
-      public void setNextReader(IndexReader reader, int docBase) {}
+      public void setNextReader(AtomicReaderContext context) {}
       @Override
       public boolean acceptsDocsOutOfOrder() {
         return true;

Modified: lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynExpand.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynExpand.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynExpand.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynExpand.java Thu Jan 13 19:53:21 2011
@@ -33,13 +33,13 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.Version;
@@ -104,7 +104,7 @@ public final class SynExpand {
 	 * @return the expanded Query
 	 */ 
 	public static Query expand( String query,
-								Searcher syns,
+								IndexSearcher syns,
 								Analyzer a,
 								String f,
 								final float boost)
@@ -161,9 +161,9 @@ public final class SynExpand {
         }
 
         @Override
-        public void setNextReader(IndexReader reader, int docBase)
+        public void setNextReader(AtomicReaderContext context)
             throws IOException {
-          this.reader = reader;
+          this.reader = context.reader;
         }
 
         @Override

Modified: lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynLookup.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynLookup.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/java/org/apache/lucene/wordnet/SynLookup.java Thu Jan 13 19:53:21 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
@@ -39,7 +40,6 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.FSDirectory;
 
@@ -60,7 +60,7 @@ public class SynLookup {
     }
 
     @Override
-    public void setNextReader(IndexReader reader, int docBase) {}
+    public void setNextReader(AtomicReaderContext context) {}
     @Override
     public boolean acceptsDocsOutOfOrder() {
       return true;
@@ -114,7 +114,7 @@ public class SynLookup {
 	 * @param boost
 	 */ 
 	public static Query expand( String query,
-								Searcher syns,
+								IndexSearcher syns,
 								Analyzer a,
 								final String field,
 								final float boost)
@@ -170,9 +170,9 @@ public class SynLookup {
         }
 
         @Override
-        public void setNextReader(IndexReader reader, int docBase)
+        public void setNextReader(AtomicReaderContext context)
             throws IOException {
-          this.reader = reader;
+          this.reader = context.reader;
         }
 
         @Override

Modified: lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/test/org/apache/lucene/wordnet/TestWordnet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/test/org/apache/lucene/wordnet/TestWordnet.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/test/org/apache/lucene/wordnet/TestWordnet.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/wordnet/src/test/org/apache/lucene/wordnet/TestWordnet.java Thu Jan 13 19:53:21 2011
@@ -26,13 +26,12 @@ import org.apache.lucene.search.BooleanC
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Searcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestWordnet extends LuceneTestCase {
-  private Searcher searcher;
+  private IndexSearcher searcher;
   private Directory dir;
   
   String storePathName = new File(TEMP_DIR,"testLuceneWordnet").getAbsolutePath();

Modified: lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/NumericRangeFilterBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/NumericRangeFilterBuilder.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/NumericRangeFilterBuilder.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/java/org/apache/lucene/xmlparser/builders/NumericRangeFilterBuilder.java Thu Jan 13 19:53:21 2011
@@ -19,7 +19,7 @@ package org.apache.lucene.xmlparser.buil
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.NumericRangeFilter;
@@ -157,7 +157,7 @@ public class NumericRangeFilterBuilder i
 		private static final long serialVersionUID = 1L;
 
 		@Override
-		public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+		public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
 			return null;
 		}
 

Modified: lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/test/org/apache/lucene/xmlparser/builders/TestNumericRangeFilterBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/test/org/apache/lucene/xmlparser/builders/TestNumericRangeFilterBuilder.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/test/org/apache/lucene/xmlparser/builders/TestNumericRangeFilterBuilder.java (original)
+++ lucene/dev/branches/realtime_search/lucene/contrib/xml-query-parser/src/test/org/apache/lucene/xmlparser/builders/TestNumericRangeFilterBuilder.java Thu Jan 13 19:53:21 2011
@@ -28,7 +28,9 @@ import javax.xml.parsers.ParserConfigura
 import org.apache.lucene.util.LuceneTestCase;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.NumericRangeFilter;
 import org.apache.lucene.store.Directory;
@@ -64,10 +66,10 @@ public class TestNumericRangeFilterBuild
 		writer.commit();
 		try
 		{
-			IndexReader reader = IndexReader.open(ramDir, true);
+			IndexReader reader = new SlowMultiReaderWrapper(IndexReader.open(ramDir, true));
 			try
 			{
-				assertNull(filter.getDocIdSet(reader));
+				assertNull(filter.getDocIdSet((AtomicReaderContext) reader.getTopReaderContext()));
 			}
 			finally
 			{

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/AbstractField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/AbstractField.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/AbstractField.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/AbstractField.java Thu Jan 13 19:53:21 2011
@@ -99,7 +99,7 @@ public abstract class AbstractField impl
    *
    * <p>Note: this value is not stored directly with the document in the index.
    * Documents returned from {@link org.apache.lucene.index.IndexReader#document(int)} and
-   * {@link org.apache.lucene.search.Searcher#doc(int)} may thus not have the same value present as when
+   * {@link org.apache.lucene.search.IndexSearcher#doc(int)} may thus not have the same value present as when
    * this field was indexed.
    *
    * @see #setBoost(float)

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Document.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Document.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Document.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Document.java Thu Jan 13 19:53:21 2011
@@ -18,8 +18,8 @@ package org.apache.lucene.document;
  */
 
 import java.util.*;             // for javadoc
+import org.apache.lucene.search.IndexSearcher;  // for javadoc
 import org.apache.lucene.search.ScoreDoc; // for javadoc
-import org.apache.lucene.search.Searcher;  // for javadoc
 import org.apache.lucene.index.IndexReader;  // for javadoc
 
 /** Documents are the unit of indexing and search.
@@ -32,8 +32,7 @@ import org.apache.lucene.index.IndexRead
  *
  * <p>Note that fields which are <i>not</i> {@link Fieldable#isStored() stored} are
  * <i>not</i> available in documents retrieved from the index, e.g. with {@link
- * ScoreDoc#doc}, {@link Searcher#doc(int)} or {@link
- * IndexReader#document(int)}.
+ * ScoreDoc#doc} or {@link IndexReader#document(int)}.
  */
 
 public final class Document implements java.io.Serializable {
@@ -167,7 +166,7 @@ public final class Document implements j
   /** Returns a List of all the fields in a document.
    * <p>Note that fields which are <i>not</i> {@link Fieldable#isStored() stored} are
    * <i>not</i> available in documents retrieved from the
-   * index, e.g. {@link Searcher#doc(int)} or {@link
+   * index, e.g. {@link IndexSearcher#doc(int)} or {@link
    * IndexReader#document(int)}.
    */
   public final List<Fieldable> getFields() {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Fieldable.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Fieldable.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Fieldable.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/document/Fieldable.java Thu Jan 13 19:53:21 2011
@@ -64,7 +64,7 @@ public interface Fieldable extends Seria
    *
    * <p>Note: this value is not stored directly with the document in the index.
    * Documents returned from {@link org.apache.lucene.index.IndexReader#document(int)} and
-   * {@link org.apache.lucene.search.Searcher#doc(int)} may thus not have the same value present as when
+   * {@link org.apache.lucene.search.IndexSearcher#doc(int)} may thus not have the same value present as when
    * this field was indexed.
    *
    * @see #setBoost(float)
@@ -206,7 +206,7 @@ public interface Fieldable extends Seria
   * required in the index, it also means any query
   * requiring positional information, such as {@link
   * PhraseQuery} or {@link SpanQuery} subclasses will
-  * silently fail to find results.
+  * fail with an exception.
   */
   void setOmitTermFreqAndPositions(boolean omitTermFreqAndPositions);
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/BufferedDeletes.java Thu Jan 13 19:53:21 2011
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
@@ -364,12 +365,14 @@ class BufferedDeletes {
     // Delete by query
     if (deletes.queries.size() > 0) {
       IndexSearcher searcher = new IndexSearcher(reader);
+      assert searcher.getTopReaderContext().isAtomic;
+      final AtomicReaderContext readerContext = (AtomicReaderContext) searcher.getTopReaderContext();
       try {
         for (Entry<Query, Integer> entry : deletes.queries.entrySet()) {
           Query query = entry.getKey();
           int limit = entry.getValue().intValue();
           Weight weight = query.weight(searcher);
-          Scorer scorer = weight.scorer(reader, true, false);
+          Scorer scorer = weight.scorer(readerContext, Weight.ScorerContext.def());
           if (scorer != null) {
             while(true)  {
               int doc = scorer.nextDoc();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/CheckIndex.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/CheckIndex.java Thu Jan 13 19:53:21 2011
@@ -548,10 +548,10 @@ public class CheckIndex {
       if (infoStream != null) {
         infoStream.print("    test: field norms.........");
       }
-      final byte[] b = new byte[reader.maxDoc()];
+      byte[] b;
       for (final String fieldName : fieldNames) {
         if (reader.hasNorms(fieldName)) {
-          reader.norms(fieldName, b, 0);
+          b = reader.norms(fieldName);
           ++status.totFields;
         }
       }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Thu Jan 13 19:53:21 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -31,13 +30,11 @@ import java.util.Set;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.BytesRef;
 
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
@@ -62,9 +59,8 @@ class DirectoryReader extends IndexReade
   private boolean rollbackHasChanges;
 
   private SegmentReader[] subReaders;
+  private ReaderContext topLevelReaderContext;
   private int[] starts;                           // 1st docno for each segment
-  private final Map<SegmentReader,ReaderUtil.Slice> subReaderToSlice = new HashMap<SegmentReader,ReaderUtil.Slice>();
-  private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
   private int maxDoc = 0;
   private int numDocs = -1;
   private boolean hasDeletions = false;
@@ -186,7 +182,7 @@ class DirectoryReader extends IndexReade
 
   /** This constructor is only used for {@link #reopen()} */
   DirectoryReader(Directory directory, SegmentInfos infos, SegmentReader[] oldReaders, int[] oldStarts,
-                  Map<String,byte[]> oldNormsCache, boolean readOnly, boolean doClone, int termInfosIndexDivisor, CodecProvider codecs) throws IOException {
+                  boolean readOnly, boolean doClone, int termInfosIndexDivisor, CodecProvider codecs) throws IOException {
     this.directory = directory;
     this.readOnly = readOnly;
     this.segmentInfos = infos;
@@ -274,38 +270,6 @@ class DirectoryReader extends IndexReade
     
     // initialize the readers to calculate maxDoc before we try to reuse the old normsCache
     initialize(newReaders);
-    
-    // try to copy unchanged norms from the old normsCache to the new one
-    if (oldNormsCache != null) {
-      for (Map.Entry<String,byte[]> entry: oldNormsCache.entrySet()) {
-        String field = entry.getKey();
-        if (!hasNorms(field)) {
-          continue;
-        }
-
-        byte[] oldBytes = entry.getValue();
-
-        byte[] bytes = new byte[maxDoc()];
-
-        for (int i = 0; i < subReaders.length; i++) {
-          Integer oldReaderIndex = segmentReaders.get(subReaders[i].getSegmentName());
-
-          // this SegmentReader was not re-opened, we can copy all of its norms 
-          if (oldReaderIndex != null &&
-               (oldReaders[oldReaderIndex.intValue()] == subReaders[i] 
-                 || oldReaders[oldReaderIndex.intValue()].norms.get(field) == subReaders[i].norms.get(field))) {
-            // we don't have to synchronize here: either this constructor is called from a SegmentReader,
-            // in which case no old norms cache is present, or it is called from MultiReader.reopen(),
-            // which is synchronized
-            System.arraycopy(oldBytes, oldStarts[oldReaderIndex.intValue()], bytes, starts[i], starts[i+1] - starts[i]);
-          } else {
-            subReaders[i].norms(field, bytes, starts[i]);
-          }
-        }
-
-        normsCache.put(field, bytes);      // update cache
-      }
-    }
   }
 
   /** {@inheritDoc} */
@@ -335,25 +299,22 @@ class DirectoryReader extends IndexReade
   private void initialize(SegmentReader[] subReaders) throws IOException {
     this.subReaders = subReaders;
     starts = new int[subReaders.length + 1];    // build starts array
-
+    final AtomicReaderContext[] subReaderCtx = new AtomicReaderContext[subReaders.length];
+    topLevelReaderContext = new CompositeReaderContext(this, subReaderCtx, subReaderCtx);
     final List<Fields> subFields = new ArrayList<Fields>();
-    final List<ReaderUtil.Slice> fieldSlices = new ArrayList<ReaderUtil.Slice>();
-
+    
     for (int i = 0; i < subReaders.length; i++) {
       starts[i] = maxDoc;
+      subReaderCtx[i] = new AtomicReaderContext(topLevelReaderContext, subReaders[i], i, maxDoc, i, maxDoc);
       maxDoc += subReaders[i].maxDoc();      // compute maxDocs
 
       if (subReaders[i].hasDeletions()) {
         hasDeletions = true;
       }
-
-      final ReaderUtil.Slice slice = new ReaderUtil.Slice(starts[i], subReaders[i].maxDoc(), i);
-      subReaderToSlice.put(subReaders[i], slice);
-
+      
       final Fields f = subReaders[i].fields();
       if (f != null) {
         subFields.add(f);
-        fieldSlices.add(slice);
       }
     }
     starts[subReaders.length] = maxDoc;
@@ -497,7 +458,7 @@ class DirectoryReader extends IndexReade
 
   private synchronized DirectoryReader doReopen(SegmentInfos infos, boolean doClone, boolean openReadOnly) throws CorruptIndexException, IOException {
     DirectoryReader reader;
-    reader = new DirectoryReader(directory, infos, subReaders, starts, normsCache, openReadOnly, doClone, termInfosIndexDivisor, codecs);
+    reader = new DirectoryReader(directory, infos, subReaders, starts, openReadOnly, doClone, termInfosIndexDivisor, codecs);
     return reader;
   }
 
@@ -637,41 +598,12 @@ class DirectoryReader extends IndexReade
   @Override
   public synchronized byte[] norms(String field) throws IOException {
     ensureOpen();
-    byte[] bytes = normsCache.get(field);
-    if (bytes != null)
-      return bytes;          // cache hit
-    if (!hasNorms(field))
-      return null;
-
-    bytes = new byte[maxDoc()];
-    for (int i = 0; i < subReaders.length; i++)
-      subReaders[i].norms(field, bytes, starts[i]);
-    normsCache.put(field, bytes);      // update cache
-    return bytes;
-  }
-
-  @Override
-  public synchronized void norms(String field, byte[] result, int offset)
-    throws IOException {
-    ensureOpen();
-    byte[] bytes = normsCache.get(field);
-    if (bytes==null && !hasNorms(field)) {
-      Arrays.fill(result, offset, result.length, Similarity.getDefault().encodeNormValue(1.0f));
-    } else if (bytes != null) {                           // cache hit
-      System.arraycopy(bytes, 0, result, offset, maxDoc());
-    } else {
-      for (int i = 0; i < subReaders.length; i++) {      // read from segments
-        subReaders[i].norms(field, result, offset + starts[i]);
-      }
-    }
+    throw new UnsupportedOperationException("please use MultiNorms.norms, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level norms");
   }
 
   @Override
   protected void doSetNorm(int n, String field, byte value)
     throws CorruptIndexException, IOException {
-    synchronized (normsCache) {
-      normsCache.remove(field);                         // clear cache      
-    }
     int i = readerIndex(n);                           // find segment num
     subReaders[i].setNorm(n-starts[i], field, value); // dispatch
   }
@@ -864,7 +796,6 @@ class DirectoryReader extends IndexReade
   @Override
   protected synchronized void doClose() throws IOException {
     IOException ioe = null;
-    normsCache = null;
     for (int i = 0; i < subReaders.length; i++) {
       // try to close each reader, even if an exception is thrown
       try {
@@ -903,16 +834,16 @@ class DirectoryReader extends IndexReade
       fieldSet.addAll(names);
     }
     return fieldSet;
-  } 
+  }
   
   @Override
-  public IndexReader[] getSequentialSubReaders() {
-    return subReaders;
+  public ReaderContext getTopReaderContext() {
+    return topLevelReaderContext;
   }
-
+  
   @Override
-  public int getSubReaderDocBase(IndexReader subReader) {
-    return subReaderToSlice.get(subReader).start;
+  public IndexReader[] getSequentialSubReaders() {
+    return subReaders;
   }
 
   /** Returns the directory this index resides in. */

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java Thu Jan 13 19:53:21 2011
@@ -84,7 +84,6 @@ final class DocFieldProcessor extends Do
     // FieldInfo.storePayload.
     final String fileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELD_INFOS_EXTENSION);
     fieldInfos.write(state.directory, fileName);
-    state.flushedFiles.add(fileName);
   }
 
   @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/DocumentsWriter.java Thu Jan 13 19:53:21 2011
@@ -370,7 +370,6 @@ final class DocumentsWriter {
       throws CorruptIndexException, IOException {
     ensureOpen();
 
-    Collection<String> flushedFiles = null;
     SegmentInfo newSegment = null;
 
     ThreadState perThread = perThreadPool.getAndLock(Thread.currentThread(), this, doc);
@@ -390,8 +389,6 @@ final class DocumentsWriter {
       newSegment = finishAddDocument(dwpt, perThreadRAMUsedBeforeAdd);
       if (newSegment != null) {
         perThreadPool.clearThreadBindings(perThread);
-        flushedFiles = new HashSet<String>();
-        flushedFiles.addAll(dwpt.flushState.flushedFiles);
       }
 
     } finally {
@@ -399,7 +396,7 @@ final class DocumentsWriter {
     }
 
     if (newSegment != null) {
-      finishFlushedSegment(newSegment, flushedFiles);
+      finishFlushedSegment(newSegment);
       return true;
     }
 
@@ -443,7 +440,6 @@ final class DocumentsWriter {
     boolean anythingFlushed = false;
 
     while (threadsIterator.hasNext()) {
-      Collection<String> flushedFiles = null;
       SegmentInfo newSegment = null;
 
       ThreadState perThread = threadsIterator.next();
@@ -465,8 +461,6 @@ final class DocumentsWriter {
 
           if (newSegment != null) {
             IndexWriter.setDiagnostics(newSegment, "flush");
-            flushedFiles = new HashSet<String>();
-            flushedFiles.addAll(dwpt.flushState.flushedFiles);
             dwpt.pushDeletes(newSegment, indexWriter.segmentInfos);
             anythingFlushed = true;
             perThreadPool.clearThreadBindings(perThread);
@@ -481,7 +475,7 @@ final class DocumentsWriter {
       if (newSegment != null) {
         // important do unlock the perThread before finishFlushedSegment
         // is called to prevent deadlock on IndexWriter mutex
-        finishFlushedSegment(newSegment, flushedFiles);
+        finishFlushedSegment(newSegment);
       }
     }
 
@@ -500,14 +494,14 @@ final class DocumentsWriter {
     cfsWriter.close();
   }
 
-  void finishFlushedSegment(SegmentInfo newSegment, Collection<String> flushedFiles) throws IOException {
+  void finishFlushedSegment(SegmentInfo newSegment) throws IOException {
     if (indexWriter.useCompoundFile(newSegment)) {
       String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
       message("creating compound file " + compoundFileName);
       // Now build compound file
       boolean success = false;
       try {
-        createCompoundFile(compoundFileName, flushedFiles);
+        createCompoundFile(compoundFileName, newSegment.files());
         success = true;
       } finally {
         if (!success) {
@@ -518,14 +512,14 @@ final class DocumentsWriter {
 
           indexWriter.deleter.deleteFile(IndexFileNames.segmentFileName(newSegment.name, "",
               IndexFileNames.COMPOUND_FILE_EXTENSION));
-          for (String file : flushedFiles) {
+          for (String file : newSegment.files()) {
             indexWriter.deleter.deleteFile(file);
           }
 
         }
       }
 
-      for (String file : flushedFiles) {
+      for (String file : newSegment.files()) {
         indexWriter.deleter.deleteFile(file);
       }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfo.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfo.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfo.java Thu Jan 13 19:53:21 2011
@@ -52,7 +52,7 @@ public final class FieldInfo {
       this.storeOffsetWithTermVector = false;
       this.storePositionWithTermVector = false;
       this.storePayloads = false;
-      this.omitNorms = true;
+      this.omitNorms = false;
       this.omitTermFreqAndPositions = false;
     }
   }
@@ -82,7 +82,7 @@ public final class FieldInfo {
         this.storePayloads = true;
       }
       if (this.omitNorms != omitNorms) {
-        this.omitNorms = false;                // once norms are stored, always store
+        this.omitNorms = true;                // if one require omitNorms at least once, it remains off for life
       }
       if (this.omitTermFreqAndPositions != omitTermFreqAndPositions) {
         this.omitTermFreqAndPositions = true;                // if one require omitTermFreqAndPositions at least once, it remains off for life

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfos.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInfos.java Thu Jan 13 19:53:21 2011
@@ -278,14 +278,21 @@ public final class FieldInfos {
   }
 
   public boolean hasVectors() {
-    boolean hasVectors = false;
     for (int i = 0; i < size(); i++) {
       if (fieldInfo(i).storeTermVector) {
-        hasVectors = true;
-        break;
+        return true;
       }
     }
-    return hasVectors;
+    return false;
+  }
+
+  public boolean hasNorms() {
+    for (int i = 0; i < size(); i++) {
+      if (!fieldInfo(i).omitNorms) {
+        return true;
+      }
+    }
+    return false;
   }
 
   public void write(Directory d, String name) throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInvertState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInvertState.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInvertState.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FieldInvertState.java Thu Jan 13 19:53:21 2011
@@ -73,6 +73,10 @@ public final class FieldInvertState {
     return length;
   }
 
+  public void setLength(int length) {
+    this.length = length;
+  }
+  
   /**
    * Get the number of terms with <code>positionIncrement == 0</code>.
    * @return the numOverlap
@@ -81,6 +85,10 @@ public final class FieldInvertState {
     return numOverlap;
   }
 
+  public void setNumOverlap(int numOverlap) {
+    this.numOverlap = numOverlap;
+  }
+  
   /**
    * Get end offset of the last processed term.
    * @return the offset
@@ -99,6 +107,10 @@ public final class FieldInvertState {
     return boost;
   }
   
+  public void setBoost(float boost) {
+    this.boost = boost;
+  }
+  
   public AttributeSource getAttributeSource() {
     return attributeSource;
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Fields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Fields.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Fields.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Fields.java Thu Jan 13 19:53:21 2011
@@ -28,7 +28,7 @@ public abstract class Fields {
    *  names.  This will not return null.  */
   public abstract FieldsEnum iterator() throws IOException;
 
-  /** Get the {@link Terms} for this field.  This may return
+  /** Get the {@link Terms} for this field.  This will return
    *  null if the field does not exist. */
   public abstract Terms terms(String field) throws IOException;
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Thu Jan 13 19:53:21 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
@@ -130,11 +131,6 @@ public class FilterIndexReader extends I
     }
 
     @Override
-    public void cacheCurrentTerm() throws IOException {
-      in.cacheCurrentTerm();
-    }
-
-    @Override
     public SeekStatus seek(long ord) throws IOException {
       return in.seek(ord);
     }
@@ -173,6 +169,16 @@ public class FilterIndexReader extends I
     public Comparator<BytesRef> getComparator() throws IOException {
       return in.getComparator();
     }
+
+    @Override
+    public SeekStatus seek(BytesRef term, TermState state) throws IOException {
+      return in.seek(term, state);
+    }
+
+    @Override
+    public TermState termState() throws IOException {
+      return in.termState();
+    }
   }
 
   /** Base class for filtering {@link DocsEnum} implementations. */
@@ -350,12 +356,6 @@ public class FilterIndexReader extends I
   }
 
   @Override
-  public void norms(String f, byte[] bytes, int offset) throws IOException {
-    ensureOpen();
-    in.norms(f, bytes, offset);
-  }
-
-  @Override
   protected void doSetNorm(int d, String f, byte b) throws CorruptIndexException, IOException {
     in.setNorm(d, f, b);
   }
@@ -417,6 +417,11 @@ public class FilterIndexReader extends I
   public IndexReader[] getSequentialSubReaders() {
     return in.getSequentialSubReaders();
   }
+  
+  @Override
+  public ReaderContext getTopReaderContext() {
+    return in.getTopReaderContext();
+  }
 
   @Override
   public Fields fields() throws IOException {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexReader.java Thu Jan 13 19:53:21 2011
@@ -301,7 +301,7 @@ public abstract class IndexReader implem
    * @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
    * @param termInfosIndexDivisor Subsamples which indexed
    *  terms are loaded into RAM. This has the same effect as {@link
-   *  IndexWriter#setTermIndexInterval} except that setting
+   *  IndexWriterConfig#setTermIndexInterval} except that setting
    *  must be done at indexing time while this setting can be
    *  set per reader.  When set to N, then one in every
    *  N*termIndexInterval terms in the index is loaded into
@@ -351,7 +351,7 @@ public abstract class IndexReader implem
    * @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
    * @param termInfosIndexDivisor Subsamples which indexed
    *  terms are loaded into RAM. This has the same effect as {@link
-   *  IndexWriter#setTermIndexInterval} except that setting
+   *  IndexWriterConfig#setTermIndexInterval} except that setting
    *  must be done at indexing time while this setting can be
    *  set per reader.  When set to N, then one in every
    *  N*termIndexInterval terms in the index is loaded into
@@ -380,7 +380,7 @@ public abstract class IndexReader implem
    * @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
    * @param termInfosIndexDivisor Subsamples which indexed
    *  terms are loaded into RAM. This has the same effect as {@link
-   *  IndexWriter#setTermIndexInterval} except that setting
+   *  IndexWriterConfig#setTermIndexInterval} except that setting
    *  must be done at indexing time while this setting can be
    *  set per reader.  When set to N, then one in every
    *  N*termIndexInterval terms in the index is loaded into
@@ -413,7 +413,7 @@ public abstract class IndexReader implem
    * @param readOnly true if no changes (deletions, norms) will be made with this IndexReader
    * @param termInfosIndexDivisor Subsamples which indexed
    *  terms are loaded into RAM. This has the same effect as {@link
-   *  IndexWriter#setTermIndexInterval} except that setting
+   *  IndexWriterConfig#setTermIndexInterval} except that setting
    *  must be done at indexing time while this setting can be
    *  set per reader.  When set to N, then one in every
    *  N*termIndexInterval terms in the index is loaded into
@@ -542,7 +542,7 @@ public abstract class IndexReader implem
    * file descriptors, CPU time) will be consumed.</p>
    *
    * <p>For lower latency on reopening a reader, you should
-   * call {@link #setMergedSegmentWarmer} to
+   * call {@link IndexWriterConfig#setMergedSegmentWarmer} to
    * pre-warm a newly merged segment before it's committed
    * to the index.  This is important for minimizing
    * index-to-search delay after a large merge.  </p>
@@ -931,14 +931,6 @@ public abstract class IndexReader implem
    */
   public abstract byte[] norms(String field) throws IOException;
 
-  /** Reads the byte-encoded normalization factor for the named field of every
-   *  document.  This is used by the search code to score documents.
-   *
-   * @see org.apache.lucene.document.Field#setBoost(float)
-   */
-  public abstract void norms(String field, byte[] bytes, int offset)
-    throws IOException;
-
   /** Expert: Resets the normalization factor for the named field of the named
    * document.  The norm represents the product of the field's {@link
    * org.apache.lucene.document.Fieldable#setBoost(float) boost} and its {@link Similarity#lengthNorm(String,
@@ -970,26 +962,6 @@ public abstract class IndexReader implem
   protected abstract void doSetNorm(int doc, String field, byte value)
           throws CorruptIndexException, IOException;
 
-  /** Expert: Resets the normalization factor for the named field of the named
-   * document.
-   *
-   * @see #norms(String)
-   * @see Similarity#decodeNormValue(byte)
-   * 
-   * @throws StaleReaderException if the index has changed
-   *  since this reader was opened
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws LockObtainFailedException if another writer
-   *  has this index open (<code>write.lock</code> could not
-   *  be obtained)
-   * @throws IOException if there is a low-level IO error
-   */
-  public void setNorm(int doc, String field, float value)
-          throws StaleReaderException, CorruptIndexException, LockObtainFailedException, IOException {
-    ensureOpen();
-    setNorm(doc, field, Similarity.getDefault().encodeNormValue(value));
-  }
-
   /** Flex API: returns {@link Fields} for this reader.
    *  This method may return null if the reader has no
    *  postings.
@@ -1070,6 +1042,47 @@ public abstract class IndexReader implem
       return null;
     }
   }
+  
+  /**
+   * Returns {@link DocsEnum} for the specified field and
+   * {@link TermState}. This may return null, if either the field or the term
+   * does not exists or the {@link TermState} is invalid for the underlying
+   * implementation.*/
+  public DocsEnum termDocsEnum(Bits skipDocs, String field, BytesRef term, TermState state) throws IOException {
+    assert state != null;
+    assert field != null;
+    final Fields fields = fields();
+    if (fields == null) {
+      return null;
+    }
+    final Terms terms = fields.terms(field);
+    if (terms != null) {
+      return terms.docs(skipDocs, term, state, null);
+    } else {
+      return null;
+    }
+  }
+  
+  /**
+   * Returns {@link DocsAndPositionsEnum} for the specified field and
+   * {@link TermState}. This may return null, if either the field or the term
+   * does not exists, the {@link TermState} is invalid for the underlying
+   * implementation, or positions were not stored for this term.*/
+  public DocsAndPositionsEnum termPositionsEnum(Bits skipDocs, String field, BytesRef term, TermState state) throws IOException {
+    assert state != null;
+    assert field != null;
+    final Fields fields = fields();
+    if (fields == null) {
+      return null;
+    }
+    final Terms terms = fields.terms(field);
+    if (terms != null) {
+      return terms.docsAndPositions(skipDocs, term, state, null);
+    } else {
+      return null;
+    }
+  }
+
 
   /** Deletes the document numbered <code>docNum</code>.  Once a document is
    * deleted it will not appear in TermDocs or TermPositions enumerations.
@@ -1126,7 +1139,7 @@ public abstract class IndexReader implem
     if (docs == null) return 0;
     int n = 0;
     int doc;
-    while ((doc = docs.nextDoc()) != docs.NO_MORE_DOCS) {
+    while ((doc = docs.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
       deleteDocument(doc);
       n++;
     }
@@ -1356,9 +1369,7 @@ public abstract class IndexReader implem
   }
 
   /** Expert: returns the sequential sub readers that this
-   *  reader is logically composed of.  For example,
-   *  IndexSearcher uses this API to drive searching by one
-   *  sub reader at a time.  If this reader is not composed
+   *  reader is logically composed of. If this reader is not composed
    *  of sequential child readers, it should return null.
    *  If this method returns an empty array, that means this
    *  reader is a null reader (for example a MultiReader
@@ -1373,12 +1384,33 @@ public abstract class IndexReader implem
   public IndexReader[] getSequentialSubReaders() {
     return null;
   }
-
-
-  /** Expert: returns the docID base for this subReader. */
-  public int getSubReaderDocBase(IndexReader subReader) {
-    throw new UnsupportedOperationException();
-  }
+  
+  /**
+   * Expert: Returns a the root {@link ReaderContext} for this
+   * {@link IndexReader}'s sub-reader tree. Iff this reader is composed of sub
+   * readers ,ie. this reader being a composite reader, this method returns a
+   * {@link CompositeReaderContext} holding the reader's direct children as well as a
+   * view of the reader tree's atomic leaf contexts. All sub-
+   * {@link ReaderContext} instances referenced from this readers top-level
+   * context are private to this reader and are not shared with another context
+   * tree. For example, IndexSearcher uses this API to drive searching by one
+   * atomic leaf reader at a time. If this reader is not composed of child
+   * readers, this method returns an {@link AtomicReaderContext}.
+   * <p>
+   * Note: Any of the sub-{@link CompositeReaderContext} instances reference from this
+   * top-level context holds a <code>null</code> {@link CompositeReaderContext#leaves}
+   * reference. Only the top-level context maintains the convenience leaf-view
+   * for performance reasons.
+   * <p>
+   * NOTE: You should not try using sub-readers returned by this method to make
+   * any changes (setNorm, deleteDocument, etc.). While this might succeed for
+   * one composite reader (like MultiReader), it will most likely lead to index
+   * corruption for other readers (like DirectoryReader obtained through
+   * {@link #open}. Use the top-level context's reader directly.
+   * 
+   * @lucene.experimental
+   */
+  public abstract ReaderContext getTopReaderContext();
 
   /** Expert */
   public Object getCoreCacheKey() {
@@ -1431,4 +1463,132 @@ public abstract class IndexReader implem
   Fields retrieveFields() {
     return fields;
   }
+
+  /**
+   * A struct like class that represents a hierarchical relationship between
+   * {@link IndexReader} instances. 
+   * @lucene.experimental
+   */
+  public static abstract class ReaderContext {
+    /** The reader context for this reader's immediate parent, or null if none */
+    public final ReaderContext parent;
+    /** The actual reader */
+    public final IndexReader reader;
+    /** <code>true</code> iff the reader is an atomic reader */
+    public final boolean isAtomic;
+    /** <code>true</code> if this context struct represents the top level reader within the hierarchical context */
+    public final boolean isTopLevel;
+    /** the doc base for this reader in the parent, <tt>0</tt> if parent is null */
+    public final int docBaseInParent;
+    /** the ord for this reader in the parent, <tt>0</tt> if parent is null */
+    public final int ordInParent;
+    
+    ReaderContext(ReaderContext parent, IndexReader reader,
+        boolean isAtomic, int ordInParent, int docBaseInParent) {
+      this.parent = parent;
+      this.reader = reader;
+      this.isAtomic = isAtomic;
+      this.docBaseInParent = docBaseInParent;
+      this.ordInParent = ordInParent;
+      this.isTopLevel = parent==null;
+    }
+    
+    /**
+     * Returns the context's leaves if this context is a top-level context
+     * otherwise <code>null</code>.
+     * <p>
+     * Note: this is convenience method since leaves can always be obtained by
+     * walking the context tree.
+     */
+    public AtomicReaderContext[] leaves() {
+      return null;
+    }
+    
+    /**
+     * Returns the context's children iff this context is a composite context
+     * otherwise <code>null</code>.
+     * <p>
+     * Note: this method is a convenience method to prevent
+     * <code>instanceof</code> checks and type-casts to
+     * {@link CompositeReaderContext}.
+     */
+    public ReaderContext[] children() {
+      return null;
+    }
+  }
+  
+  /**
+   * {@link ReaderContext} for composite {@link IndexReader} instance.
+   * @lucene.experimental
+   */
+  public static final class CompositeReaderContext extends ReaderContext {
+    /** the composite readers immediate children */
+    public final ReaderContext[] children;
+    /** the composite readers leaf reader contexts if this is the top level reader in this context */
+    public final AtomicReaderContext[] leaves;
+
+    /**
+     * Creates a {@link CompositeReaderContext} for intermediate readers that aren't
+     * not top-level readers in the current context
+     */
+    public CompositeReaderContext(ReaderContext parent, IndexReader reader,
+        int ordInParent, int docbaseInParent, ReaderContext[] children) {
+      this(parent, reader, ordInParent, docbaseInParent, children, null);
+    }
+    
+    /**
+     * Creates a {@link CompositeReaderContext} for top-level readers with parent set to <code>null</code>
+     */
+    public CompositeReaderContext(IndexReader reader, ReaderContext[] children, AtomicReaderContext[] leaves) {
+      this(null, reader, 0, 0, children, leaves);
+    }
+    
+    private CompositeReaderContext(ReaderContext parent, IndexReader reader,
+        int ordInParent, int docbaseInParent, ReaderContext[] children,
+        AtomicReaderContext[] leaves) {
+      super(parent, reader, false, ordInParent, docbaseInParent);
+      this.children = children;
+      this.leaves = leaves;
+    }
+
+    @Override
+    public AtomicReaderContext[] leaves() {
+      return leaves;
+    }
+    
+    
+    @Override
+    public ReaderContext[] children() {
+      return children;
+    }
+  }
+  
+  /**
+   * {@link ReaderContext} for atomic {@link IndexReader} instances
+   * @lucene.experimental
+   */
+  public static final class AtomicReaderContext extends ReaderContext {
+    /** The readers ord in the top-level's leaves array */
+    public final int ord;
+    /** The readers absolute doc base */
+    public final int docBase;
+    /**
+     * Creates a new {@link AtomicReaderContext} 
+     */    
+    public AtomicReaderContext(ReaderContext parent, IndexReader reader,
+        int ord, int docBase, int leafOrd, int leafDocBase) {
+      super(parent, reader, true, ord, docBase);
+      assert reader.getSequentialSubReaders() == null : "Atomic readers must not have subreaders";
+      this.ord = leafOrd;
+      this.docBase = leafDocBase;
+    }
+    
+    /**
+     * Creates a new {@link AtomicReaderContext} for a atomic reader without an immediate
+     * parent.
+     */
+    public AtomicReaderContext(IndexReader atomicReader) {
+      this(null, atomicReader, 0, 0, 0, 0);
+    }
+  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriter.java Thu Jan 13 19:53:21 2011
@@ -2250,11 +2250,11 @@ public class IndexWriter implements Clos
       // Now create the compound file if needed
       if (useCompoundFile) {
         merger.createCompoundFile(mergedName + ".cfs", info);
-        info.setUseCompoundFile(true);
 
         // delete new non cfs files directly: they were never
         // registered with IFD
-        deleter.deleteNewFiles(merger.getMergedFiles(info));
+        deleter.deleteNewFiles(info.files());
+        info.setUseCompoundFile(true);
       }
 
       // Register the new segment
@@ -3129,7 +3129,7 @@ public class IndexWriter implements Clos
 
             synchronized(this) {
               deleter.deleteFile(compoundFileName);
-              deleter.deleteNewFiles(merger.getMergedFiles(merge.info));
+              deleter.deleteNewFiles(merge.info.files());
             }
           }
         }
@@ -3140,7 +3140,7 @@ public class IndexWriter implements Clos
 
           // delete new non cfs files directly: they were never
           // registered with IFD
-          deleter.deleteNewFiles(merger.getMergedFiles(merge.info));
+          deleter.deleteNewFiles(merge.info.files());
 
           if (merge.isAborted()) {
             if (infoStream != null) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java Thu Jan 13 19:53:21 2011
@@ -55,7 +55,7 @@ public final class IndexWriterConfig imp
   public static enum OpenMode { CREATE, APPEND, CREATE_OR_APPEND }
 
   /** Default value is 32. Change using {@link #setTermIndexInterval(int)}. */
-  public static final int DEFAULT_TERM_INDEX_INTERVAL = 32;
+  public static final int DEFAULT_TERM_INDEX_INTERVAL = 32;                   // TODO: this should be private to the codec, not settable here
 
   /** Denotes a flush trigger is disabled. */
   public final static int DISABLE_AUTO_FLUSH = -1;
@@ -115,7 +115,7 @@ public final class IndexWriterConfig imp
   private OpenMode openMode;
   private int maxFieldLength;
   private Similarity similarity;
-  private int termIndexInterval;
+  private int termIndexInterval; // TODO: this should be private to the codec, not settable here
   private MergeScheduler mergeScheduler;
   private long writeLockTimeout;
   private int maxBufferedDeleteTerms;
@@ -147,7 +147,7 @@ public final class IndexWriterConfig imp
     openMode = OpenMode.CREATE_OR_APPEND;
     maxFieldLength = UNLIMITED_FIELD_LENGTH;
     similarity = Similarity.getDefault();
-    termIndexInterval = DEFAULT_TERM_INDEX_INTERVAL;
+    termIndexInterval = DEFAULT_TERM_INDEX_INTERVAL; // TODO: this should be private to the codec, not settable here
     mergeScheduler = new ConcurrentMergeScheduler();
     writeLockTimeout = WRITE_LOCK_TIMEOUT;
     maxBufferedDeleteTerms = DEFAULT_MAX_BUFFERED_DELETE_TERMS;
@@ -312,7 +312,7 @@ public final class IndexWriterConfig imp
    *
    * @see #DEFAULT_TERM_INDEX_INTERVAL
    */
-  public IndexWriterConfig setTermIndexInterval(int interval) {
+  public IndexWriterConfig setTermIndexInterval(int interval) { // TODO: this should be private to the codec, not settable here
     this.termIndexInterval = interval;
     return this;
   }
@@ -322,7 +322,7 @@ public final class IndexWriterConfig imp
    *
    * @see #setTermIndexInterval(int)
    */
-  public int getTermIndexInterval() {
+  public int getTermIndexInterval() { // TODO: this should be private to the codec, not settable here
     return termIndexInterval;
   }
 
@@ -617,7 +617,7 @@ public final class IndexWriterConfig imp
     sb.append("openMode=").append(openMode).append("\n");
     sb.append("maxFieldLength=").append(maxFieldLength).append("\n");
     sb.append("similarity=").append(similarity.getClass().getName()).append("\n");
-    sb.append("termIndexInterval=").append(termIndexInterval).append("\n");
+    sb.append("termIndexInterval=").append(termIndexInterval).append("\n"); // TODO: this should be private to the codec, not settable here
     sb.append("mergeScheduler=").append(mergeScheduler.getClass().getName()).append("\n");
     sb.append("default WRITE_LOCK_TIMEOUT=").append(WRITE_LOCK_TIMEOUT).append("\n");
     sb.append("writeLockTimeout=").append(writeLockTimeout).append("\n");

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1058718&r1=1058717&r2=1058718&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java Thu Jan 13 19:53:21 2011
@@ -241,6 +241,9 @@ public abstract class LogMergePolicy ext
     while (start >= 0) {
       SegmentInfo info = infos.info(start);
       if (size(info) > maxMergeSize || sizeDocs(info) > maxMergeDocs) {
+        if (verbose()) {
+          message("optimize: skip segment=" + info + ": size is > maxMergeSize (" + maxMergeSize + ") or sizeDocs is > maxMergeDocs (" + maxMergeDocs + ")");
+        }
         // need to skip that segment + add a merge for the 'right' segments,
         // unless there is only 1 which is optimized.
         if (last - start - 1 > 1 || (start != last - 1 && !isOptimized(infos.info(start + 1)))) {
@@ -335,10 +338,18 @@ public abstract class LogMergePolicy ext
       int maxNumSegments, Set<SegmentInfo> segmentsToOptimize) throws IOException {
 
     assert maxNumSegments > 0;
+    if (verbose()) {
+      message("findMergesForOptimize: maxNumSegs=" + maxNumSegments + " segsToOptimize= "+ segmentsToOptimize);
+    }
 
     // If the segments are already optimized (e.g. there's only 1 segment), or
     // there are <maxNumSegements, all optimized, nothing to do.
-    if (isOptimized(infos, maxNumSegments, segmentsToOptimize)) return null;
+    if (isOptimized(infos, maxNumSegments, segmentsToOptimize)) {
+      if (verbose()) {
+        message("already optimized; skip");
+      }
+      return null;
+    }
 
     // Find the newest (rightmost) segment that needs to
     // be optimized (other segments may have been flushed
@@ -352,10 +363,20 @@ public abstract class LogMergePolicy ext
       }
     }
 
-    if (last == 0) return null;
+    if (last == 0) {
+      if (verbose()) {
+        message("last == 0; skip");
+      }
+      return null;
+    }
 
     // There is only one segment already, and it is optimized
-    if (maxNumSegments == 1 && last == 1 && isOptimized(infos.info(0))) return null;
+    if (maxNumSegments == 1 && last == 1 && isOptimized(infos.info(0))) {
+      if (verbose()) {
+        message("already 1 seg; skip");
+      }
+      return null;
+    }
 
     // Check if there are any segments above the threshold
     boolean anyTooLarge = false;