You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by mi...@apache.org on 2010/02/10 11:58:22 UTC

svn commit: r908447 [1/3] - in /lucene/java/branches/flex_1458: contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/ contrib/misc/src/java/org/apache/lucene/index/ contrib/misc/src/test/org/apache/lucene/index/ contrib/spellchecker/src/java/o...

Author: mikemccand
Date: Wed Feb 10 10:58:19 2010
New Revision: 908447

URL: http://svn.apache.org/viewvc?rev=908447&view=rev
Log:
LUCENE-2111 (flex branch): factor out Multi*Enums

Added:
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsEnum.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFields.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFieldsEnum.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTerms.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTermsEnum.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitsSlice.java   (with props)
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/MultiBits.java   (with props)
Removed:
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMergeInfo.java
Modified:
    lucene/java/branches/flex_1458/contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
    lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java
    lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java
    lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java
    lucene/java/branches/flex_1458/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java
    lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java
    lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java
    lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java
    lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverterPerField.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Fields.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMerger.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Terms.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHashPerField.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MergeState.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/PostingsConsumer.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/TermsConsumer.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/AutomatonQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheImpl.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PhraseQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/Spans.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/TermSpans.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Bits.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/OpenBitSet.java
    lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/ReaderUtil.java
    lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestExternalCodecs.java
    lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
    lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestIndexWriter.java

Modified: lucene/java/branches/flex_1458/contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java (original)
+++ lucene/java/branches/flex_1458/contrib/benchmark/src/test/org/apache/lucene/benchmark/byTask/TestPerfTasksLogic.java Wed Feb 10 10:58:19 2010
@@ -37,6 +37,7 @@
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.SerialMergeScheduler;
@@ -469,7 +470,7 @@
 
     int totalTokenCount2 = 0;
 
-    FieldsEnum fields = reader.fields().iterator();
+    FieldsEnum fields = MultiFields.getFields(reader).iterator();
     String fieldName = null;
     while((fieldName = fields.next()) != null) {
       if (fieldName == DocMaker.ID_FIELD)

Modified: lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java (original)
+++ lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/FieldNormModifier.java Wed Feb 10 10:58:19 2010
@@ -19,12 +19,15 @@
 import java.io.IOException;
 import java.io.File;
 import java.util.Date;
+import java.util.List;
+import java.util.ArrayList;
 
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.ReaderUtil;
 
 /**
  * Given a directory and a list of fields, updates the fieldNorms in place for every document.
@@ -105,43 +108,46 @@
    */
   public void reSetNorms(String field) throws IOException {
     String fieldName = StringHelper.intern(field);
-    int[] termCounts = new int[0];
-    byte[] fakeNorms = new byte[0];
     
     IndexReader reader = null;
     try {
-      reader = IndexReader.open(dir, true);
-      final Bits delDocs = reader.getDeletedDocs();
+      reader = IndexReader.open(dir, false);
 
-        termCounts = new int[reader.maxDoc()];
-        Terms terms = reader.fields().terms(field);
-        if (terms != null) {
-          TermsEnum termsEnum = terms.iterator();
-          DocsEnum docs = null;
-          while(termsEnum.next() != null) {
-            docs = termsEnum.docs(delDocs, docs);
-            while(true) {
-              int docID = docs.nextDoc();
-              if (docID != docs.NO_MORE_DOCS) {
-                termCounts[docID] += docs.freq();
-              } else {
-                break;
+      final List<IndexReader> subReaders = new ArrayList<IndexReader>();
+      ReaderUtil.gatherSubReaders(subReaders, reader);
+
+      for(IndexReader subReader : subReaders) {
+        final Bits delDocs = subReader.getDeletedDocs();
+
+        int[] termCounts = new int[subReader.maxDoc()];
+        Fields fields = subReader.fields();
+        if (fields != null) {
+          Terms terms = fields.terms(field);
+          if (terms != null) {
+            TermsEnum termsEnum = terms.iterator();
+            DocsEnum docs = null;
+            while(termsEnum.next() != null) {
+              docs = termsEnum.docs(delDocs, docs);
+              while(true) {
+                int docID = docs.nextDoc();
+                if (docID != docs.NO_MORE_DOCS) {
+                  termCounts[docID] += docs.freq();
+                } else {
+                  break;
+                }
               }
             }
           }
         }
-    } finally {
-      if (null != reader) reader.close();
-    }
-    
-    try {
-      reader = IndexReader.open(dir, false); 
-      for (int d = 0; d < termCounts.length; d++) {
-        if (! reader.isDeleted(d)) {
-          if (sim == null)
-            reader.setNorm(d, fieldName, fakeNorms[0]);
-          else
-            reader.setNorm(d, fieldName, sim.encodeNormValue(sim.lengthNorm(fieldName, termCounts[d])));
+
+        for (int d = 0; d < termCounts.length; d++) {
+          if (delDocs == null || !delDocs.get(d)) {
+            if (sim == null) {
+              subReader.setNorm(d, fieldName, Similarity.encodeNorm(1.0f));
+            } else {
+              subReader.setNorm(d, fieldName, sim.encodeNormValue(sim.lengthNorm(fieldName, termCounts[d])));
+            }
+          }
         }
       }
       

Modified: lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java (original)
+++ lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/MultiPassIndexSplitter.java Wed Feb 10 10:58:19 2010
@@ -170,6 +170,8 @@
    * list of deletions.
    */
   public static class FakeDeleteIndexReader extends FilterIndexReader {
+    // nocommit -- implement flex api here
+
     OpenBitSet dels;
     OpenBitSet oldDels = null;
 
@@ -213,6 +215,11 @@
     }
 
     @Override
+    public IndexReader[] getSequentialSubReaders() {
+      return null;
+    }
+
+    @Override
     public Bits getDeletedDocs() {
       return dels;
     }

Modified: lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java (original)
+++ lucene/java/branches/flex_1458/contrib/misc/src/java/org/apache/lucene/index/TermVectorAccessor.java Wed Feb 10 10:58:19 2010
@@ -102,7 +102,7 @@
 
     final Bits delDocs = indexReader.getDeletedDocs();
 
-    Terms terms = indexReader.fields().terms(field);
+    Terms terms = MultiFields.getTerms(indexReader, field);
     boolean anyTerms = false;
     if (terms != null) {
       TermsEnum termsEnum = terms.iterator();

Modified: lucene/java/branches/flex_1458/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java (original)
+++ lucene/java/branches/flex_1458/contrib/misc/src/test/org/apache/lucene/index/TestFieldNormModifier.java Wed Feb 10 10:58:19 2010
@@ -76,13 +76,9 @@
     writer.close();
   }
   
-  public void testMissingField() {
+  public void testMissingField() throws Exception {
     FieldNormModifier fnm = new FieldNormModifier(store, s);
-    try {
-      fnm.reSetNorms("nobodyherebutuschickens");
-    } catch (Exception e) {
-      assertNull("caught something", e);
-    }
+    fnm.reSetNorms("nobodyherebutuschickens");
   }
   
   public void testFieldWithNoNorm() throws Exception {
@@ -97,11 +93,7 @@
     r.close();
     
     FieldNormModifier fnm = new FieldNormModifier(store, s);
-    try {
-      fnm.reSetNorms("nonorm");
-    } catch (Exception e) {
-      assertNull("caught something", e);
-    }
+    fnm.reSetNorms("nonorm");
     
     // nothing should have changed
     r = IndexReader.open(store, false);

Modified: lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java (original)
+++ lucene/java/branches/flex_1458/contrib/spellchecker/src/java/org/apache/lucene/search/spell/LuceneDictionary.java Wed Feb 10 10:58:19 2010
@@ -24,6 +24,7 @@
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.StringHelper;
 
 import java.io.*;
@@ -58,7 +59,7 @@
 
     LuceneIterator() {
       try {
-        Terms terms = reader.fields().terms(field);
+        final Terms terms = MultiFields.getTerms(reader, field);
         if (terms != null) {
           termsEnum = terms.iterator();
           pendingTerm = termsEnum.next();

Modified: lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndPrefixQuery.java Wed Feb 10 10:58:19 2010
@@ -21,6 +21,7 @@
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
 
 import java.io.IOException;
 
@@ -57,7 +58,7 @@
     MatchingTermVisitor mtv) throws IOException
   {
     /* inspired by PrefixQuery.rewrite(): */
-    Terms terms = reader.fields().terms(fieldName);
+    Terms terms = MultiFields.getTerms(reader, fieldName);
     boolean expanded = false;
     if (terms != null) {
       TermsEnum termsEnum = terms.iterator();

Modified: lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTermQuery.java Wed Feb 10 10:58:19 2010
@@ -22,6 +22,7 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.BytesRef;
 
  
@@ -48,7 +49,7 @@
     MatchingTermVisitor mtv) throws IOException
   {
     /* check term presence in index here for symmetry with other SimpleTerm's */
-    Terms terms = reader.fields().terms(fieldName);
+    Terms terms = MultiFields.getTerms(reader, fieldName);
     if (terms != null) {
       TermsEnum termsEnum = terms.iterator();
 

Modified: lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java (original)
+++ lucene/java/branches/flex_1458/contrib/surround/src/java/org/apache/lucene/queryParser/surround/query/SrndTruncQuery.java Wed Feb 10 10:58:19 2010
@@ -21,6 +21,7 @@
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
 
 import java.io.IOException;
 
@@ -89,7 +90,7 @@
   {
     boolean expanded = false;
     int prefixLength = prefix.length();
-    Terms terms = reader.fields().terms(fieldName);
+    Terms terms = MultiFields.getTerms(reader, fieldName);
     if (terms != null) {
       Matcher matcher = pattern.matcher("");
       try {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DirectoryReader.java Wed Feb 10 10:58:19 2010
@@ -36,10 +36,10 @@
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.index.codecs.Codecs;
-import org.apache.lucene.util.PriorityQueue;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.MultiBits;
 
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
 
@@ -66,14 +66,12 @@
 
   private SegmentReader[] subReaders;
   private int[] starts;                           // 1st docno for each segment
-  private final Map<SegmentReader,Integer> subReaderToDocBase = new HashMap<SegmentReader,Integer>();
+  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;
 
-  private MultiFields fields;
-
 //  static IndexReader open(final Directory directory, final IndexDeletionPolicy deletionPolicy, final IndexCommit commit, final boolean readOnly,
 //      final int termInfosIndexDivisor) throws CorruptIndexException, IOException {
 //    return open(directory, deletionPolicy, commit, readOnly, termInfosIndexDivisor, null);
@@ -359,19 +357,31 @@
     return buffer.toString();
   }
 
-  private void initialize(SegmentReader[] subReaders) {
+  private void initialize(SegmentReader[] subReaders) throws IOException {
     this.subReaders = subReaders;
     starts = new int[subReaders.length + 1];    // build starts array
     Bits[] subs = new Bits[subReaders.length];
+
+    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;
       maxDoc += subReaders[i].maxDoc();      // compute maxDocs
 
       if (subReaders[i].hasDeletions()) {
         hasDeletions = true;
+        subs[i] = subReaders[i].getDeletedDocs();
+      }
+
+      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);
       }
-      subs[i] = subReaders[i].getDeletedDocs();
-      subReaderToDocBase.put(subReaders[i], Integer.valueOf(starts[i]));
     }
     starts[subReaders.length] = maxDoc;
 
@@ -380,66 +390,13 @@
     } else {
       deletedDocs = null;
     }
-
-    fields = new MultiFields(subReaders, starts);
   }
 
-  private MultiBits deletedDocs;
-
-  // Exposes a slice of an existing Bits as a new Bits.
-  // Only used when one provides an external skipDocs (ie,
-  // not the del docs from this DirectoryReader), to pull
-  // the DocsEnum of the sub readers
-  private final static class SubBits implements Bits {
-    private final Bits parent;
-    private final int start;
-    private final int length;
-
-    // start is inclusive; end is exclusive (length = end-start)
-    public SubBits(Bits parent, int start, int length) {
-      this.parent = parent;
-      this.start = start;
-      this.length = length;
-      assert length >= 0: "length=" + length;
-    }
-    
-    public boolean get(int doc) {
-      if (doc >= length) {
-        throw new RuntimeException("doc " + doc + " is out of bounds 0 .. " + (length-1));
-      }
-      assert doc < length: "doc=" + doc + " length=" + length;
-      return parent.get(doc+start);
-    }
-  }
-    
-  // Concatenates multiple Bits together
-  // nocommit -- if none of the subs have deletions we
-  // should return null from getDeletedDocs:
-  static final class MultiBits implements Bits {
-    private final Bits[] subs;
-    // this is 1+subs.length, ie the last entry has the maxDoc
-    final int[] starts;
-
-    public MultiBits(Bits[] subs, int[] starts) {
-      this.subs = subs;
-      this.starts = starts;
-    }
-
-    public boolean get(int doc) {
-      final int reader = ReaderUtil.subIndex(doc, starts);
-      final Bits bits = subs[reader];
-      if (bits == null) {
-        return false;
-      } else {
-        final int length = starts[1+reader]-starts[reader];
-        assert doc - starts[reader] < length: "doc=" + doc + " reader=" + reader + " starts[reader]=" + starts[reader] + " length=" + length;
-        return bits.get(doc-starts[reader]);
-      }
-    }
-  }
+  private Bits deletedDocs;
 
   @Override
   public Bits getDeletedDocs() {
+    // nocommit -- maybe not supported?
     return deletedDocs;
   }
 
@@ -828,11 +785,10 @@
 
   @Override
   public Fields fields() throws IOException {
-    if (subReaders.length == 1) {
-      // Optimize the single reader case
-      return subReaders[0].fields();
+    if (subReaders.length == 0) {
+      return null;
     } else {
-      return fields;
+      throw new UnsupportedOperationException("please use MultiFields.getFields if you really need a top level Fields for this reader");
     }
   }
 
@@ -901,6 +857,11 @@
    */
   @Override
   protected void doCommit(Map<String,String> commitUserData) throws IOException {
+    // poll subreaders for changes
+    for (int i = 0; !hasChanges && i < subReaders.length; i++) {
+      hasChanges |= subReaders[i].hasChanges;
+    }
+    
     if (hasChanges) {
       segmentInfos.setUserData(commitUserData);
       // Default deleter (for backwards compatibility) is
@@ -1051,7 +1012,7 @@
 
   @Override
   public int getSubReaderDocBase(IndexReader subReader) {
-    return subReaderToDocBase.get(subReader).intValue();
+    return subReaderToSlice.get(subReader).start;
   }
 
   /** Returns the directory this index resides in. */
@@ -1187,572 +1148,6 @@
     }
   }
   
-  private final static class TermsWithBase {
-    Terms terms;
-    int base;
-    int length;
-    Bits skipDocs;
-
-    public TermsWithBase(IndexReader reader, int base, String field) throws IOException {
-      this.base = base;
-      length = reader.maxDoc();
-      assert length >= 0: "length=" + length;
-      skipDocs = reader.getDeletedDocs();
-      terms = reader.fields().terms(field);
-    }
-  }
-
-  private final static class FieldsEnumWithBase {
-    FieldsEnum fields;
-    String current;
-    int base;
-    int length;
-    Bits skipDocs;
-
-    public FieldsEnumWithBase(IndexReader reader, int base) throws IOException {
-      this.base = base;
-      length = reader.maxDoc();
-      assert length >= 0: "length=" + length;
-      skipDocs = reader.getDeletedDocs(); 
-      fields = reader.fields().iterator();
-    }
-  }
-
-  private final static class TermsEnumWithBase {
-    final TermsEnum terms;
-    final int base;
-    final int length;
-    BytesRef current;
-    final Bits skipDocs;
-
-    public TermsEnumWithBase(FieldsEnumWithBase start, TermsEnum terms, BytesRef term) {
-      this.terms = terms;
-      current = term;
-      skipDocs = start.skipDocs;
-      base = start.base;
-      length = start.length;
-      assert length >= 0: "length=" + length;
-    }
-
-    public TermsEnumWithBase(TermsWithBase start, TermsEnum terms, BytesRef term) {
-      this.terms = terms;
-      current = term;
-      skipDocs = start.skipDocs;
-      base = start.base;
-      length = start.length;
-      assert length >= 0: "length=" + length;
-    }
-  }
-
-  private final static class PostingsEnumWithBase {
-    DocsAndPositionsEnum postings;
-    int base;
-  }
-
-  private final static class FieldMergeQueue extends PriorityQueue<FieldsEnumWithBase> {
-    FieldMergeQueue(int size) {
-      initialize(size);
-    }
-
-    @Override
-    protected final boolean lessThan(FieldsEnumWithBase fieldsA, FieldsEnumWithBase fieldsB) {
-      return fieldsA.current.compareTo(fieldsB.current) < 0;
-    }
-  }
-
-  private final static class TermMergeQueue extends PriorityQueue<TermsEnumWithBase> {
-    BytesRef.Comparator termComp;
-    TermMergeQueue(int size) {
-      initialize(size);
-    }
-
-    @Override
-    protected final boolean lessThan(TermsEnumWithBase termsA, TermsEnumWithBase termsB) {
-      final int cmp = termComp.compare(termsA.current, termsB.current);
-      if (cmp != 0) {
-        return cmp < 0;
-      } else {
-        return termsA.base < termsB.base;
-      }
-    }
-  }
-
-  // Exposes flex API, merged from flex API of
-  // sub-segments.
-  final static class MultiFields extends Fields {
-    private final IndexReader[] readers;
-    private final int[] starts;
-    private final HashMap<String,MultiTerms> terms = new HashMap<String,MultiTerms>();
-
-    public MultiFields(IndexReader[] readers, int[] starts) {
-      this.readers = readers;
-      this.starts = starts;
-    }
-
-    @Override
-    public FieldsEnum iterator() throws IOException {
-      FieldsEnumWithBase[] subs = new FieldsEnumWithBase[readers.length];
-      for(int i=0;i<subs.length;i++) {
-        subs[i] = new FieldsEnumWithBase(readers[i], starts[i]);
-      }
-      return new MultiFieldsEnum(subs);
-    }
-
-    @Override
-    public Terms terms(String field) throws IOException {
-      MultiTerms result = terms.get(field);
-      if (result == null) {
-
-        // First time this field is requested, we create & add to terms:
-        List<TermsWithBase> subs = new ArrayList<TermsWithBase>();
-
-        // Gather all sub-readers that share this field
-        for(int i=0;i<readers.length;i++) {
-          Terms subTerms = readers[i].fields().terms(field);
-          if (subTerms != null) {
-            subs.add(new TermsWithBase(readers[i], starts[i], field));
-          }
-        }
-        result = new MultiTerms(subs.toArray(new TermsWithBase[]{}));
-        terms.put(field, result);
-      }
-      return result;
-    }
-  }
-
-  // Exposes flex API, merged from flex API of
-  // sub-segments.
-  private final static class MultiTerms extends Terms {
-    private final TermsWithBase[] subs;
-    private final BytesRef.Comparator termComp;
-
-    public MultiTerms(TermsWithBase[] subs) throws IOException {
-      this.subs = subs;
-      
-      BytesRef.Comparator _termComp = null;
-      for(int i=0;i<subs.length;i++) {
-        if (_termComp == null) {
-          _termComp = subs[i].terms.getComparator();
-        } else {
-          // We cannot merge sub-readers that have
-          // different TermComps
-          final BytesRef.Comparator subTermComp = subs[i].terms.getComparator();
-          if (subTermComp != null && !subTermComp.equals(_termComp)) {
-            throw new IllegalStateException("sub-readers have different BytesRef.Comparators; cannot merge");
-          }
-        }
-      }
-      termComp = _termComp;
-    }
-
-    @Override
-    public TermsEnum iterator() throws IOException {
-      return new MultiTermsEnum(subs.length).reset(subs);
-    }
-
-    @Override
-    public BytesRef.Comparator getComparator() {
-      return termComp;
-    }
-  }
-
-  // Exposes flex API, merged from flex API of
-  // sub-segments.  This does a merge sort, by field name,
-  // of the sub-readers.
-  private final static class MultiFieldsEnum extends FieldsEnum {
-    private final FieldMergeQueue queue;
-
-    // Holds sub-readers containing field we are currently
-    // on, popped from queue.
-    private final FieldsEnumWithBase[] top;
-    private int numTop;
-
-    // Re-used TermsEnum
-    private final MultiTermsEnum terms;
-
-    private String currentField;
-    
-    MultiFieldsEnum(FieldsEnumWithBase[] subs) throws IOException {
-      terms = new MultiTermsEnum(subs.length);
-      queue = new FieldMergeQueue(subs.length);
-      top = new FieldsEnumWithBase[subs.length];
-
-      // Init q
-      for(int i=0;i<subs.length;i++) {
-        subs[i].current = subs[i].fields.next();
-        if (subs[i].current != null) {
-          queue.add(subs[i]);
-        }
-      }
-    }
-
-    public String field() {
-      assert currentField != null;
-      assert numTop > 0;
-      return currentField;
-    }
-
-    @Override
-    public String next() throws IOException {
-
-      // restore queue
-      for(int i=0;i<numTop;i++) {
-        top[i].current = top[i].fields.next();
-        if (top[i].current != null) {
-          queue.add(top[i]);
-        } else {
-          // no more fields in this sub-reader
-        }
-      }
-
-      numTop = 0;
-
-      // gather equal top fields
-      if (queue.size() > 0) {
-        while(true) {
-          top[numTop++] = queue.pop();
-          if (queue.size() == 0 || (queue.top()).current != top[0].current) {
-            break;
-          }
-        }
-        currentField = top[0].current;
-      } else {
-        currentField = null;
-      }
-
-      return currentField;
-    }
-
-    @Override
-    public TermsEnum terms() throws IOException {
-      return terms.reset(top, numTop);
-    }
-  }
-
-  // Exposes flex API, merged from flex API of
-  // sub-segments.  This does a merge sort, by term text, of
-  // the sub-readers.
-  private static final class MultiTermsEnum extends TermsEnum {
-    
-    private final TermMergeQueue queue;
-    private final TermsEnumWithBase[] subs;
-    private final TermsEnumWithBase[] top;
-    int numTop;
-    int numSubs;
-    private BytesRef current;
-    private BytesRef.Comparator termComp;
-
-    MultiTermsEnum(int size) {
-      queue = new TermMergeQueue(size);
-      top = new TermsEnumWithBase[size];
-      subs = new TermsEnumWithBase[size];
-    }
-
-    @Override
-    public BytesRef term() {
-      return current;
-    }
-
-    @Override
-    public BytesRef.Comparator getComparator() {
-      return termComp;
-    }
-
-    MultiTermsEnum reset(TermsWithBase[] terms) throws IOException {
-      assert terms.length <= top.length;
-      numSubs = 0;
-      numTop = 0;
-      termComp = null;
-      queue.clear();
-      for(int i=0;i<terms.length;i++) {
-        final TermsEnum termsEnum = terms[i].terms.iterator();
-        if (termsEnum != null) {
-          if (termComp == null) {
-            queue.termComp = termComp = termsEnum.getComparator();
-          } else {
-            // We cannot merge sub-readers that have
-            // different TermComps
-            final BytesRef.Comparator subTermComp = termsEnum.getComparator();
-            if (subTermComp != null && !subTermComp.equals(termComp)) {
-              throw new IllegalStateException("sub-readers have different BytesRef.Comparators; cannot merge");
-            }
-          }
-          final BytesRef term = termsEnum.next();
-          if (term != null) {
-            subs[numSubs] = new TermsEnumWithBase(terms[i], termsEnum, term);
-            queue.add(subs[numSubs]);
-            numSubs++;
-          } else {
-            // field has no terms
-          }
-        }
-      }
-
-      return this;
-    }
-
-    MultiTermsEnum reset(FieldsEnumWithBase[] fields, int numFields) throws IOException {
-      assert numFields <= top.length;
-      numSubs = 0;
-      numTop = 0;
-      termComp = null;
-      queue.clear();
-      for(int i=0;i<numFields;i++) {
-        final TermsEnum terms = fields[i].fields.terms();
-        if (terms != null) {
-          final BytesRef term = terms.next();
-          if (term != null) {
-            if (termComp == null) {
-              queue.termComp = termComp = terms.getComparator();
-            } else {
-              assert termComp.equals(terms.getComparator());
-            }
-            subs[numSubs] = new TermsEnumWithBase(fields[i], terms, term);
-            queue.add(subs[numSubs]);
-            numSubs++;
-          } else {
-            // field has no terms
-          }
-        }
-      }
-
-      return this;
-    }
-
-    @Override
-    public SeekStatus seek(BytesRef term) throws IOException {
-      queue.clear();
-      numTop = 0;
-      for(int i=0;i<numSubs;i++) {
-        final SeekStatus status = subs[i].terms.seek(term);
-        if (status == SeekStatus.FOUND) {
-          top[numTop++] = subs[i];
-          subs[i].current = term;
-        } else if (status == SeekStatus.NOT_FOUND) {
-          subs[i].current = subs[i].terms.term();
-          assert subs[i].current != null;
-          queue.add(subs[i]);
-        } else {
-          // enum exhausted
-        }
-      }
-
-      if (numTop > 0) {
-        current = term;
-        return SeekStatus.FOUND;
-      } else if (queue.size() > 0) {
-        pullTop();
-        return SeekStatus.NOT_FOUND;
-      } else {
-        return SeekStatus.END;
-      }
-    }
-
-    @Override
-    public SeekStatus seek(long ord) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public long ord() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    private final void pullTop() {
-      // extract all subs from the queue that have the same
-      // top term
-      assert numTop == 0;
-      while(true) {
-        top[numTop++] = queue.pop();
-        if (queue.size() == 0 || !(queue.top()).current.bytesEquals(top[0].current)) {
-          break;
-        }
-      } 
-      current = top[0].current;
-    }
-
-    private final void pushTop() throws IOException {
-      // call next() on each top, and put back into queue
-      for(int i=0;i<numTop;i++) {
-        top[i].current = top[i].terms.next();
-        if (top[i].current != null) {
-          queue.add(top[i]);
-        } else {
-          // no more fields in this reader
-        }
-      }
-      numTop = 0;
-    }
-
-    @Override
-    public BytesRef next() throws IOException {
-      // restore queue
-      pushTop();
-
-      // gather equal top fields
-      if (queue.size() > 0) {
-        pullTop();
-      } else {
-        current = null;
-      }
-
-      return current;
-    }
-
-    @Override
-    public int docFreq() {
-      int sum = 0;
-      for(int i=0;i<numTop;i++) {
-        sum += top[i].terms.docFreq();
-      }
-      return sum;
-    }
-
-    @Override
-    public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
-      return docsAndPositions(skipDocs, (DocsAndPositionsEnum) reuse);
-    }
-
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
-      if (reuse != null) {
-        return ((MultiDocsAndPositionsEnum) reuse).reset(top, numTop, skipDocs);
-      } else {
-        return new MultiDocsAndPositionsEnum(subs.length).reset(top, numTop, skipDocs);
-      }
-    }
-  }
-
-  private static final class MultiDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    final PostingsEnumWithBase[] subs;
-    int numSubs;
-    int upto;
-    DocsAndPositionsEnum currentDocs;
-    int currentBase;
-    Bits skipDocs;
-    int doc = -1;
-
-    MultiDocsAndPositionsEnum(int count) {
-      subs = new PostingsEnumWithBase[count];
-    }
-
-    MultiDocsAndPositionsEnum reset(TermsEnumWithBase[] subs, final int numSubs, final Bits skipDocs) throws IOException {
-      this.numSubs = 0;
-      this.skipDocs = skipDocs;
-      for(int i=0;i<numSubs;i++) {
-        Bits bits = null;
-        boolean handled = false;
-
-        assert subs[i].length >= 0: "subs[" + i + " of " + numSubs + "].length=" + subs[i].length;
-
-        // Optimize for common case: requested skip docs is
-        // simply our (DiretoryReader's) deleted docs.  In
-        // this case, we just pull the skipDocs from the sub
-        // reader, rather than making the inefficient
-        // Sub(Multi(sub-readers)):
-        if (skipDocs instanceof MultiBits) {
-          MultiBits multiBits = (MultiBits) skipDocs;
-          int reader = ReaderUtil.subIndex(subs[i].base, multiBits.starts);
-          assert reader < multiBits.starts.length-1: " reader=" + reader + " multiBits.starts.length=" + multiBits.starts.length;
-          final int length = multiBits.starts[reader+1] - multiBits.starts[reader];
-          if (multiBits.starts[reader] == subs[i].base &&
-              length == subs[i].length) {
-            bits = multiBits.subs[reader];
-            handled = true;
-          }
-        }
-
-        if (!handled && skipDocs != null) {
-          // custom case: requested skip docs is foreign
-          bits = new SubBits(skipDocs, subs[i].base, subs[i].length);
-        }
-
-        final DocsAndPositionsEnum postings = subs[i].terms.docsAndPositions(bits, null);
-        if (postings != null) {
-          this.subs[this.numSubs] = new PostingsEnumWithBase();
-          this.subs[this.numSubs].postings = postings;
-          this.subs[this.numSubs].base = subs[i].base;
-          this.numSubs++;
-        }
-      }
-      upto = -1;
-      currentDocs = null;
-      return this;
-    }
-
-    @Override
-    public int freq() {
-      return currentDocs.freq();
-    }
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      while(true) {
-        if (currentDocs != null) {
-          final int doc = currentDocs.advance(target-currentBase);
-          if (doc == NO_MORE_DOCS) {
-            currentDocs = null;
-          } else {
-            return this.doc = doc + currentBase;
-          }
-        } else if (upto == numSubs-1) {
-          return this.doc = NO_MORE_DOCS;
-        } else {
-          upto++;
-          currentDocs = subs[upto].postings;
-          currentBase = subs[upto].base;
-        }
-      }
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      while(true) {
-        if (currentDocs == null) {
-          if (upto == numSubs-1) {
-            return this.doc = NO_MORE_DOCS;
-          } else {
-            upto++;
-            currentDocs = subs[upto].postings;
-            currentBase = subs[upto].base;
-          }
-        }
-
-        final int doc = currentDocs.nextDoc();
-        if (doc != NO_MORE_DOCS) {
-          return this.doc = currentBase + doc;
-        } else {
-          currentDocs = null;
-        }
-      }
-    }
-
-    @Override
-    public int nextPosition() throws IOException {
-      return currentDocs.nextPosition();
-    }
-
-    @Override
-    public int getPayloadLength() {
-      return currentDocs.getPayloadLength();
-    }
-
-    @Override
-    public BytesRef getPayload() throws IOException {
-      return currentDocs.getPayload();
-    }
-
-    @Override
-    public boolean hasPayload() {
-      return currentDocs.hasPayload();
-    }
-  }
-
   // @deprecated This is pre-flex API
   // Exposes pre-flex API by doing on-the-fly merging
   // pre-flex API to each segment

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverterPerField.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverterPerField.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocInverterPerField.java Wed Feb 10 10:58:19 2010
@@ -116,8 +116,9 @@
               reader = readerValue;
             else {
               String stringValue = field.stringValue();
-              if (stringValue == null)
+              if (stringValue == null) {
                 throw new IllegalArgumentException("field must have either TokenStream, String or Reader value");
+              }
               perThread.stringReader.init(stringValue);
               reader = perThread.stringReader;
             }
@@ -132,8 +133,6 @@
           final int startLength = fieldState.length;
           
           try {
-            int offsetEnd = fieldState.offset-1;
-            
             boolean hasMoreTokens = stream.incrementToken();
 
             fieldState.attributeSource = stream;
@@ -178,7 +177,6 @@
                   docState.docWriter.setAborting();
               }
               fieldState.position++;
-              offsetEnd = fieldState.offset + offsetAttribute.endOffset();
               if (++fieldState.length >= maxFieldLength) {
                 if (docState.infoStream != null)
                   docState.infoStream.println("maxFieldLength " +maxFieldLength+ " reached for field " + fieldInfo.name + ", ignoring following tokens");

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Fields.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Fields.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Fields.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Fields.java Wed Feb 10 10:58:19 2010
@@ -24,6 +24,8 @@
 
 public abstract class Fields {
 
+  public final static Fields[] EMPTY_ARRAY = new Fields[0];
+
   /** Returns an iterator that will step through all fields
    *  names */
   public abstract FieldsEnum iterator() throws IOException;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsEnum.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldsEnum.java Wed Feb 10 10:58:19 2010
@@ -28,6 +28,8 @@
 
 public abstract class FieldsEnum {
 
+  public final static FieldsEnum[] EMPTY_ARRAY = new FieldsEnum[0];
+
   private AttributeSource atts = null;
 
   /**

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java Wed Feb 10 10:58:19 2010
@@ -882,7 +882,17 @@
   @Deprecated
   public abstract TermEnum terms() throws IOException;
 
-  // Default impl emulates new API using old one
+  /** Flex API: returns {@link Fields} for this reader.
+   *  This may return  null if there are no fields.
+   *
+   * <p><b>NOTE</b>: if this is a multi reader ({@link
+   * #getSequentialSubReaders} is not null) then this
+   * method will throw UnsupportedOperationException.  If
+   * you really need a {@link Fields} for such a reader,
+   * use {@link MultiFields#getFields}.  However, for
+   * performance reasons, it's best to get all sub-readers
+   * using {@link ReaderUtil#gatherSubReaders} and iterate
+   * through them yourself. */
   public Fields fields() throws IOException {
     return new LegacyFields(this);
   }
@@ -940,6 +950,15 @@
     return termDocs;
   }
 
+  public Terms terms(String field) throws IOException {
+    final Fields fields = fields();
+    if (fields != null) {
+      return fields.terms(field);
+    } else {
+      return null;
+    }
+  }
+
   /** Returns {@link DocsEnum} for the specified field &
    *  term.  This may return null, for example if either the
    *  field or term does not exist. */
@@ -947,24 +966,22 @@
 
     assert field != null;
     assert term != null;
-
-    final Terms terms = fields().terms(field);
-    if (terms != null) {
-      if (Codec.DEBUG) {
-        System.out.println("ir.termDocsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this);
-      }
-      final DocsEnum docs = terms.docs(skipDocs, term, null);
-      if (Codec.DEBUG) {
-        System.out.println("ir.termDocsEnum field=" + field + " docs=" +docs);
-      }
-      if (docs != null) {
+    final Fields fields = fields();
+    if (fields != null) {
+      final Terms terms = fields.terms(field);
+      if (terms != null) {
+        if (Codec.DEBUG) {
+          System.out.println("ir.termDocsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this);
+        }
+        final DocsEnum docs = terms.docs(skipDocs, term, null);
+        if (Codec.DEBUG) {
+          System.out.println("ir.termDocsEnum field=" + field + " docs=" +docs);
+        }
         return docs;
-      } else {
-        return null;
       }
-    } else {
-      return null;
     }
+
+    return null;
   }
 
   /** Returns {@link DocsAndPositionsEnum} for the specified
@@ -975,19 +992,22 @@
     assert field != null;
     assert term != null;
 
-    final Terms terms = fields().terms(field);
-    if (terms != null) {
-      if (Codec.DEBUG) {
-        System.out.println("ir.termPositionsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this);
-      }
-      final DocsAndPositionsEnum postings = terms.docsAndPositions(skipDocs, term, null);
-      if (Codec.DEBUG) {
-        System.out.println("ir.termPositionsEnum field=" + field + " postings=" +postings);
+    final Fields fields = fields();
+    if (fields != null) {
+      final Terms terms = fields.terms(field);
+      if (terms != null) {
+        if (Codec.DEBUG) {
+          System.out.println("ir.termPositionsEnum field=" + field + " term=" + term + " terms=" + terms + " this=" + this);
+        }
+        final DocsAndPositionsEnum postings = terms.docsAndPositions(skipDocs, term, null);
+        if (Codec.DEBUG) {
+          System.out.println("ir.termPositionsEnum field=" + field + " postings=" +postings);
+        }
+        return postings;
       }
-      return postings;
-    } else {
-      return null;
     }
+
+    return null;
   }
 
   /** Returns an unpositioned {@link TermDocs} enumerator.
@@ -1166,9 +1186,7 @@
    * @throws IOException if there is a low-level IO error
    */
   public final synchronized void commit(Map<String, String> commitUserData) throws IOException {
-    if (hasChanges) {
-      doCommit(commitUserData);
-    }
+    doCommit(commitUserData);
     hasChanges = false;
   }
 
@@ -1209,6 +1227,9 @@
     public boolean get(int docID) {
       return isDeleted(docID);
     }
+    public int length() {
+      return maxDoc();
+    }
   }
 
   /**
@@ -1409,4 +1430,17 @@
   public int getTermInfosIndexDivisor() {
     throw new UnsupportedOperationException("This reader does not support this method.");
   }
+
+
+  private Fields fields;
+
+  /** lucene.experimental */
+  public void storeFields(Fields fields) {
+    this.fields = fields;
+  }
+
+  /** lucene.experimental */
+  public Fields retrieveFields() {
+    return fields;
+  }
 }

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,135 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.BytesRef;
+
+import java.io.IOException;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments.
+ *
+ * @lucene.experimental
+ */
+
+public final class MultiDocsAndPositionsEnum extends DocsAndPositionsEnum {
+  private EnumWithSlice[] subs;
+  int numSubs;
+  int upto;
+  DocsAndPositionsEnum current;
+  int currentBase;
+  Bits skipDocs;
+  int doc = -1;
+
+  MultiDocsAndPositionsEnum reset(final EnumWithSlice[] subs, final int numSubs) throws IOException {
+    this.numSubs = numSubs;
+    this.subs = subs;
+    upto = -1;
+    current = null;
+    return this;
+  }
+
+  public int getNumSubs() {
+    return numSubs;
+  }
+
+  public EnumWithSlice[] getSubs() {
+    return subs;
+  }
+
+  @Override
+  public int freq() {
+    return current.freq();
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    while(true) {
+      if (current != null) {
+        final int doc = current.advance(target-currentBase);
+        if (doc == NO_MORE_DOCS) {
+          current = null;
+        } else {
+          return this.doc = doc + currentBase;
+        }
+      } else if (upto == numSubs-1) {
+        return this.doc = NO_MORE_DOCS;
+      } else {
+        upto++;
+        current = subs[upto].docsAndPositionsEnum;
+        currentBase = subs[upto].slice.start;
+      }
+    }
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    while(true) {
+      if (current == null) {
+        if (upto == numSubs-1) {
+          return this.doc = NO_MORE_DOCS;
+        } else {
+          upto++;
+          current = subs[upto].docsAndPositionsEnum;
+          currentBase = subs[upto].slice.start;
+        }
+      }
+
+      final int doc = current.nextDoc();
+      if (doc != NO_MORE_DOCS) {
+        return this.doc = currentBase + doc;
+      } else {
+        current = null;
+      }
+    }
+  }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return current.nextPosition();
+  }
+
+  @Override
+  public int getPayloadLength() {
+    return current.getPayloadLength();
+  }
+
+  @Override
+  public boolean hasPayload() {
+    return current.hasPayload();
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return current.getPayload();
+  }
+
+  // TODO: implement bulk read more efficiently than super
+  public final static class EnumWithSlice {
+    public DocsAndPositionsEnum docsAndPositionsEnum;
+    public ReaderUtil.Slice slice;
+  }
+}
+

Propchange: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsAndPositionsEnum.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsEnum.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsEnum.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsEnum.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,113 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.ReaderUtil;
+import java.io.IOException;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments.
+ *
+ * @lucene.experimental
+ */
+
+public final class MultiDocsEnum extends DocsEnum {
+  private EnumWithSlice[] subs;
+  int numSubs;
+  int upto;
+  DocsEnum current;
+  int currentBase;
+  Bits skipDocs;
+  int doc = -1;
+
+  MultiDocsEnum reset(final EnumWithSlice[] subs, final int numSubs) throws IOException {
+    this.numSubs = numSubs;
+    this.subs = subs;
+    upto = -1;
+    current = null;
+    return this;
+  }
+
+  public int getNumSubs() {
+    return numSubs;
+  }
+
+  public EnumWithSlice[] getSubs() {
+    return subs;
+  }
+
+  @Override
+  public int freq() {
+    return current.freq();
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    while(true) {
+      if (current != null) {
+        final int doc = current.advance(target-currentBase);
+        if (doc == NO_MORE_DOCS) {
+          current = null;
+        } else {
+          return this.doc = doc + currentBase;
+        }
+      } else if (upto == numSubs-1) {
+        return this.doc = NO_MORE_DOCS;
+      } else {
+        upto++;
+        current = subs[upto].docsEnum;
+        currentBase = subs[upto].slice.start;
+      }
+    }
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    while(true) {
+      if (current == null) {
+        if (upto == numSubs-1) {
+          return this.doc = NO_MORE_DOCS;
+        } else {
+          upto++;
+          current = subs[upto].docsEnum;
+          currentBase = subs[upto].slice.start;
+        }
+      }
+
+      final int doc = current.nextDoc();
+      if (doc != NO_MORE_DOCS) {
+        return this.doc = currentBase + doc;
+      } else {
+        current = null;
+      }
+    }
+  }
+
+  // TODO: implement bulk read more efficiently than super
+  public final static class EnumWithSlice {
+    public DocsEnum docsEnum;
+    public ReaderUtil.Slice slice;
+  }
+}
+

Propchange: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiDocsEnum.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFields.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFields.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFields.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,200 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ArrayList;
+import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.index.codecs.Codec;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments.
+ * This is useful when you're interacting with an {@link
+ * IndexReader} implementation that consists of sequential
+ * sub-readers (eg {@link DirectoryReade} or {@link
+ * MultiReader}).
+ *
+ * <p><b>NOTE</b>: for multi readers, you'll get better
+ * performance by gathering the sub readers using {@link
+ * ReaderUtil#gatherSubReaders} and then operate per-reader,
+ * instead of using this class.
+ *
+ * @lucene.experimental
+ */
+
+public final class MultiFields extends Fields {
+  private final Fields[] subs;
+  private final ReaderUtil.Slice[] subSlices;
+  private final Map<String,MultiTerms> terms = new HashMap<String,MultiTerms>();
+
+  public static Fields getFields(IndexReader r) throws IOException {
+    final IndexReader[] subs = r.getSequentialSubReaders();
+    if (subs == null) {
+      // already an atomic reader
+      return r.fields();
+    } else if (subs.length == 1) {
+      return getFields(subs[0]);
+    } else {
+
+      Fields currentFields = r.retrieveFields();
+      if (currentFields == null) {
+      
+        final List<Fields> fields = new ArrayList<Fields>();
+        final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
+        ReaderUtil.gatherSubFields(null, fields, slices, r, 0);
+
+        if (fields.size() == 0) {
+          return null;
+        } else if (fields.size() == 1) {
+          currentFields = fields.get(0);
+        } else {
+          currentFields = new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
+                                         slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
+        }
+        r.storeFields(currentFields);
+      }
+      return currentFields;
+    }
+  }
+
+  public static Terms getTerms(IndexReader r, String field) throws IOException {
+    final Fields fields = getFields(r);
+    if (fields != null) {
+      return fields.terms(field);
+    } else {
+      return null;
+    }
+  }
+
+  /** Returns {@link DocsEnum} for the specified field &
+   *  term.  This may return null, for example if either the
+   *  field or term does not exist. */
+  public static DocsEnum getTermDocsEnum(IndexReader r, Bits skipDocs, String field, BytesRef term) throws IOException {
+
+    assert field != null;
+    assert term != null;
+    final Fields fields = getFields(r);
+    if (fields != null) {
+      final Terms terms = fields.terms(field);
+      if (terms != null) {
+        if (Codec.DEBUG) {
+          System.out.println("mf.termDocsEnum field=" + field + " term=" + term + " terms=" + terms);
+        }
+        final DocsEnum docs = terms.docs(skipDocs, term, null);
+        if (Codec.DEBUG) {
+          System.out.println("mf.termDocsEnum field=" + field + " docs=" +docs);
+        }
+        return docs;
+      }
+    }
+
+    return null;
+  }
+
+  /** Returns {@link DocsAndPositionsEnum} for the specified
+   *  field & term.  This may return null, for example if
+   *  either the field or term does not exist. */
+  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits skipDocs, String field, BytesRef term) throws IOException {
+    assert field != null;
+    assert term != null;
+
+    final Fields fields = getFields(r);
+    if (fields != null) {
+      final Terms terms = fields.terms(field);
+      if (terms != null) {
+        if (Codec.DEBUG) {
+          System.out.println("mf.termPositionsEnum field=" + field + " term=" + term + " terms=" + terms);
+        }
+        final DocsAndPositionsEnum postings = terms.docsAndPositions(skipDocs, term, null);
+        if (Codec.DEBUG) {
+          System.out.println("mf.termPositionsEnum field=" + field + " postings=" +postings);
+        }
+        return postings;
+      }
+    }
+
+    return null;
+  }
+
+
+  public MultiFields(Fields[] subs, ReaderUtil.Slice[] subSlices) {
+    this.subs = subs;
+    this.subSlices = subSlices;
+  }
+
+  @Override
+  public FieldsEnum iterator() throws IOException {
+
+    final List<FieldsEnum> fieldsEnums = new ArrayList<FieldsEnum>();
+    final List<ReaderUtil.Slice> fieldsSlices = new ArrayList<ReaderUtil.Slice>();
+    for(int i=0;i<subs.length;i++) {
+      final FieldsEnum subFieldsEnum = subs[i].iterator();
+      if (subFieldsEnum != null) {
+        fieldsEnums.add(subFieldsEnum);
+        fieldsSlices.add(subSlices[i]);
+      }
+    }
+    if (fieldsEnums.size() == 0) {
+      return null;
+    } else {
+      return new MultiFieldsEnum(fieldsEnums.toArray(FieldsEnum.EMPTY_ARRAY),
+                                 fieldsSlices.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
+    }
+  }
+
+  @Override
+  public Terms terms(String field) throws IOException {
+
+    final MultiTerms result;
+
+    if (!terms.containsKey(field)) {
+
+      // Lazy init: first time this field is requested, we
+      // create & add to terms:
+      final List<Terms> subs2 = new ArrayList<Terms>();
+      final List<ReaderUtil.Slice> slices2 = new ArrayList<ReaderUtil.Slice>();
+
+      // Gather all sub-readers that share this field
+      for(int i=0;i<subs.length;i++) {
+        Terms subTerms = subs[i].terms(field);
+        if (subTerms != null) {
+          subs2.add(subTerms);
+          slices2.add(subSlices[i]);
+        }
+      }
+      if (subs2.size() == 0) {
+        result = null;
+        terms.put(field, null);
+      } else {
+        result = new MultiTerms(subs2.toArray(Terms.EMPTY_ARRAY),
+                                slices2.toArray(ReaderUtil.Slice.EMPTY_ARRAY));
+      }
+      terms.put(field, result);
+    } else {
+      result = terms.get(field);
+    }
+
+    return result;
+  }
+}
+

Propchange: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFields.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFieldsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFieldsEnum.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFieldsEnum.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFieldsEnum.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,142 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.ReaderUtil;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments.
+ * This does a merge sort, by field name, of the
+ * sub-readers.
+ *
+ * @lucene.experimental
+ */
+
+public final  class MultiFieldsEnum extends FieldsEnum {
+  private final FieldMergeQueue queue;
+
+  // Holds sub-readers containing field we are currently
+  // on, popped from queue.
+  private final FieldsEnumWithSlice[] top;
+  private int numTop;
+
+  // Re-used TermsEnum
+  private final MultiTermsEnum terms;
+
+  private String currentField;
+
+  /** The subs array must be newly initialized FieldsEnum
+   *  (ie, {@link FieldsEnum#next} has not been called. */
+  public MultiFieldsEnum(FieldsEnum[] subs, ReaderUtil.Slice[] subSlices) throws IOException {
+    terms = new MultiTermsEnum(subSlices);
+    queue = new FieldMergeQueue(subs.length);
+    top = new FieldsEnumWithSlice[subs.length];
+
+    // Init q
+    for(int i=0;i<subs.length;i++) {
+      assert subs[i] != null;
+      final String field = subs[i].next();
+      if (field != null) {
+        // this FieldsEnum has at least one field
+        final FieldsEnumWithSlice sub = new FieldsEnumWithSlice(subs[i], subSlices[i], i);
+        sub.current = field;
+        queue.add(sub);
+      }
+    }
+  }
+
+  @Override
+  public String next() throws IOException {
+
+    // restore queue
+    for(int i=0;i<numTop;i++) {
+      top[i].current = top[i].fields.next();
+      if (top[i].current != null) {
+        queue.add(top[i]);
+      } else {
+        // no more fields in this sub-reader
+      }
+    }
+
+    numTop = 0;
+
+    // gather equal top fields
+    if (queue.size() > 0) {
+      while(true) {
+        top[numTop++] = queue.pop();
+        if (queue.size() == 0 || (queue.top()).current != top[0].current) {
+          break;
+        }
+      }
+      currentField = top[0].current;
+    } else {
+      currentField = null;
+    }
+
+    return currentField;
+  }
+
+  @Override
+  public TermsEnum terms() throws IOException {
+    final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<MultiTermsEnum.TermsEnumIndex>();
+    for(int i=0;i<numTop;i++) {
+      final TermsEnum terms = top[i].fields.terms();
+      if (terms != null) {
+        termsEnums.add(new MultiTermsEnum.TermsEnumIndex(terms, top[i].index));
+      }
+    }
+
+    if (termsEnums.size() == 0) {
+      return null;
+    } else {
+      return terms.reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
+    }
+  }
+
+  public final static class FieldsEnumWithSlice {
+    final FieldsEnum fields;
+    final ReaderUtil.Slice slice;
+    final int index;
+    String current;
+
+    public FieldsEnumWithSlice(FieldsEnum fields, ReaderUtil.Slice slice, int index) throws IOException {
+      this.slice = slice;
+      this.index = index;
+      assert slice.length >= 0: "length=" + slice.length;
+      this.fields = fields;
+    }
+  }
+
+  private final static class FieldMergeQueue extends PriorityQueue<FieldsEnumWithSlice> {
+    FieldMergeQueue(int size) {
+      initialize(size);
+    }
+
+    @Override
+    protected final boolean lessThan(FieldsEnumWithSlice fieldsA, FieldsEnumWithSlice fieldsB) {
+      // No need to break ties by field name: TermsEnum handles that
+      return fieldsA.current.compareTo(fieldsB.current) < 0;
+    }
+  }
+}
+

Propchange: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiFieldsEnum.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java Wed Feb 10 10:58:19 2010
@@ -25,29 +25,29 @@
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.index.DirectoryReader.MultiBits;
-import org.apache.lucene.index.DirectoryReader.MultiFields;
-import org.apache.lucene.index.DirectoryReader.MultiTermDocs;
-import org.apache.lucene.index.DirectoryReader.MultiTermEnum;
-import org.apache.lucene.index.DirectoryReader.MultiTermPositions;
+import org.apache.lucene.index.DirectoryReader.MultiTermDocs;       // deprecated
+import org.apache.lucene.index.DirectoryReader.MultiTermEnum;       // deprecated
+import org.apache.lucene.index.DirectoryReader.MultiTermPositions;  // deprecated
 import org.apache.lucene.search.Similarity;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
+import org.apache.lucene.util.MultiBits;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ReaderUtil;
 
 /** An IndexReader which reads multiple indexes, appending
- * their content. */
+ *  their content. */
 public class MultiReader extends IndexReader implements Cloneable {
   protected IndexReader[] subReaders;
   private int[] starts;                           // 1st docno for each segment
-  private final Map<IndexReader,Integer> subReaderToDocBase = new HashMap<IndexReader,Integer>();
+  private final Map<IndexReader,ReaderUtil.Slice> subReaderToSlice = new HashMap<IndexReader,ReaderUtil.Slice>();
   private boolean[] decrefOnClose;                // remember which subreaders to decRef on close
   private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
   private int maxDoc = 0;
   private int numDocs = -1;
   private boolean hasDeletions = false;
   private MultiBits deletedDocs;
-  private MultiFields fields;
+  //private Fields fields;
   
  /**
   * <p>Construct a MultiReader aggregating the named set of (sub)readers.
@@ -79,6 +79,7 @@
     starts = new int[subReaders.length + 1];    // build starts array
     decrefOnClose = new boolean[subReaders.length];
     Bits[] subs = new Bits[subReaders.length];
+
     for (int i = 0; i < subReaders.length; i++) {
       starts[i] = maxDoc;
       maxDoc += subReaders[i].maxDoc();      // compute maxDocs
@@ -94,7 +95,11 @@
         hasDeletions = true;
       }
       subs[i] = subReaders[i].getDeletedDocs();
-      subReaderToDocBase.put(subReaders[i], Integer.valueOf(starts[i]));
+
+      final ReaderUtil.Slice slice = new ReaderUtil.Slice(starts[i],
+                                                          subReaders[i].maxDoc(),
+                                                          i);
+      subReaderToSlice.put(subReaders[i], slice);
     }
 
     starts[subReaders.length] = maxDoc;
@@ -103,22 +108,16 @@
     } else {
       deletedDocs = null;
     }
-    fields = new MultiFields(subReaders, starts);
   }
 
   @Override
   public int getSubReaderDocBase(IndexReader subReader) {
-    return subReaderToDocBase.get(subReader).intValue();
+    return subReaderToSlice.get(subReader).start;
   }
 
   @Override
   public Fields fields() throws IOException {
-    if (subReaders.length == 1) {
-      // Optimize the single reader case
-      return subReaders[0].fields();
-    } else {
-      return fields;
-    }
+    throw new UnsupportedOperationException("please use MultiFields.getFields if you really need a top level Fields for this reader");
   }
 
   /**

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTerms.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTerms.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTerms.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTerms.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,84 @@
+package org.apache.lucene.index;
+
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ReaderUtil;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.ArrayList;
+
+/**
+ * Exposes flex API, merged from flex API of
+ * sub-segments.
+ *
+ * @lucene.experimental
+ */
+
+public final class MultiTerms extends Terms {
+  private final Terms[] subs;
+  private final ReaderUtil.Slice[] subSlices;
+  private final BytesRef.Comparator termComp;
+
+  public MultiTerms(Terms[] subs, ReaderUtil.Slice[] subSlices) throws IOException {
+    this.subs = subs;
+    this.subSlices = subSlices;
+    
+    BytesRef.Comparator _termComp = null;
+    for(int i=0;i<subs.length;i++) {
+      if (_termComp == null) {
+        _termComp = subs[i].getComparator();
+      } else {
+        // We cannot merge sub-readers that have
+        // different TermComps
+        final BytesRef.Comparator subTermComp = subs[i].getComparator();
+        if (subTermComp != null && !subTermComp.equals(_termComp)) {
+          throw new IllegalStateException("sub-readers have different BytesRef.Comparators; cannot merge");
+        }
+      }
+    }
+
+    termComp = _termComp;
+  }
+
+  @Override
+  public TermsEnum iterator() throws IOException {
+
+    final List<MultiTermsEnum.TermsEnumIndex> termsEnums = new ArrayList<MultiTermsEnum.TermsEnumIndex>();
+    for(int i=0;i<subs.length;i++) {
+      final TermsEnum termsEnum = subs[i].iterator();
+      if (termsEnum != null) {
+        termsEnums.add(new MultiTermsEnum.TermsEnumIndex(termsEnum, i));
+      }
+    }
+
+    if (termsEnums.size() > 0) {
+      return new MultiTermsEnum(subSlices).reset(termsEnums.toArray(MultiTermsEnum.TermsEnumIndex.EMPTY_ARRAY));
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public BytesRef.Comparator getComparator() {
+    return termComp;
+  }
+}
+

Propchange: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTerms.java
------------------------------------------------------------------------------
    svn:eol-style = native