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 [2/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...

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTermsEnum.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiTermsEnum.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,384 @@
+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.BytesRef;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BitsSlice;
+import org.apache.lucene.util.MultiBits;
+import org.apache.lucene.util.ReaderUtil;
+
+import java.io.IOException;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments.
+ * This does a merge sort, by term text, of the sub-readers.
+ *
+ * @lucene.experimental
+ */
+public final class MultiTermsEnum extends TermsEnum {
+    
+  private final TermMergeQueue queue;
+  private final TermsEnumWithSlice[] subs;
+  private final TermsEnumWithSlice[] currentSubs;
+  private final TermsEnumWithSlice[] top;
+  private final MultiDocsEnum.EnumWithSlice[] subDocs;
+  private final MultiDocsAndPositionsEnum.EnumWithSlice[] subDocsAndPositions;
+
+  private int numTop;
+  private int numSubs;
+  private BytesRef current;
+  private BytesRef.Comparator termComp;
+
+  public static class TermsEnumIndex {
+    public final static TermsEnumIndex[] EMPTY_ARRAY = new TermsEnumIndex[0];
+    final int subIndex;
+    final TermsEnum termsEnum;
+
+    public TermsEnumIndex(TermsEnum termsEnum, int subIndex) {
+      this.termsEnum = termsEnum;
+      this.subIndex = subIndex;
+    }
+  }
+
+  public int getMatchCount() {
+    return numTop;
+  }
+
+  public TermsEnumWithSlice[] getMatchArray() {
+    return top;
+  }
+
+  public MultiTermsEnum(ReaderUtil.Slice[] slices) {
+    queue = new TermMergeQueue(slices.length);
+    top = new TermsEnumWithSlice[slices.length];
+    subs = new TermsEnumWithSlice[slices.length];
+    subDocs = new MultiDocsEnum.EnumWithSlice[slices.length];
+    subDocsAndPositions = new MultiDocsAndPositionsEnum.EnumWithSlice[slices.length];
+    for(int i=0;i<slices.length;i++) {
+      subs[i] = new TermsEnumWithSlice(slices[i]);
+      subDocs[i] = new MultiDocsEnum.EnumWithSlice();
+      subDocs[i].slice = slices[i];
+      subDocsAndPositions[i] = new MultiDocsAndPositionsEnum.EnumWithSlice();
+      subDocsAndPositions[i].slice = slices[i];
+    }
+    currentSubs = new TermsEnumWithSlice[slices.length];
+  }
+
+  @Override
+  public BytesRef term() {
+    return current;
+  }
+
+  @Override
+  public BytesRef.Comparator getComparator() {
+    return termComp;
+  }
+
+  /** The terms array must be newly created TermsEnum, ie
+   *  {@link TermsEnum#next} has not yet been called. */
+  public MultiTermsEnum reset(TermsEnumIndex[] termsEnumsIndex) throws IOException {
+    assert termsEnumsIndex.length <= top.length;
+    numSubs = 0;
+    numTop = 0;
+    termComp = null;
+    queue.clear();
+    for(int i=0;i<termsEnumsIndex.length;i++) {
+
+      final TermsEnumIndex termsEnumIndex = termsEnumsIndex[i];
+      assert termsEnumIndex != null;
+
+      // init our term comp
+      if (termComp == null) {
+        queue.termComp = termComp = termsEnumIndex.termsEnum.getComparator();
+      } else {
+        // We cannot merge sub-readers that have
+        // different TermComps
+        final BytesRef.Comparator subTermComp = termsEnumIndex.termsEnum.getComparator();
+        if (subTermComp != null && !subTermComp.equals(termComp)) {
+          throw new IllegalStateException("sub-readers have different BytesRef.Comparators; cannot merge");
+        }
+      }
+
+      final BytesRef term = termsEnumIndex.termsEnum.next();
+      if (term != null) {
+        final TermsEnumWithSlice entry = subs[termsEnumIndex.subIndex];
+        entry.reset(termsEnumIndex.termsEnum, term);
+        queue.add(entry);
+        currentSubs[numSubs++] = entry;
+      } else {
+        // field has no terms
+      }
+    }
+
+    if (queue.size() == 0) {
+      return null;
+    } else {
+      return this;
+    }
+  }
+
+  @Override
+  public SeekStatus seek(BytesRef term) throws IOException {
+    queue.clear();
+    numTop = 0;
+    for(int i=0;i<numSubs;i++) {
+      final SeekStatus status = currentSubs[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) {
+      // at least one sub had exact match to the requested term
+      current = term;
+      return SeekStatus.FOUND;
+    } else if (queue.size() > 0) {
+      // no sub had exact match, but at least one sub found
+      // a term after the requested term -- advance to that
+      // next term:
+      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 {
+    final MultiDocsEnum docsEnum;
+    if (reuse != null) {
+      docsEnum = (MultiDocsEnum) reuse;
+    } else {
+      docsEnum = new MultiDocsEnum();
+    }
+    
+    final MultiBits multiSkipDocs;
+    if (skipDocs instanceof MultiBits) {
+      multiSkipDocs = (MultiBits) skipDocs;
+    } else {
+      multiSkipDocs = null;
+    }
+
+    int upto = 0;
+
+    for(int i=0;i<numTop;i++) {
+
+      final TermsEnumWithSlice entry = top[i];
+
+      final Bits b;
+
+      if (multiSkipDocs != null) {
+        // optimize for common case: requested skip docs is a
+        // congruent sub-slice of MultiBits: in this case, we
+        // just pull the skipDocs from the sub reader, rather
+        // than making the inefficient
+        // Slice(Multi(sub-readers)):
+        final Bits b2 = multiSkipDocs.getMatchingSub(top[i].subSlice);
+        if (b2 != null) {
+          b = b2;
+        } else {
+          // custom case: requested skip docs is foreign:
+          // must slice it on every access
+          b = new BitsSlice(skipDocs, top[i].subSlice);
+        }
+      } else {
+        // no deletions
+        b = null;
+      }
+
+      final DocsEnum subDocsEnum = entry.terms.docs(b, entry.reuseDocs);
+
+      if (subDocsEnum != null) {
+        entry.reuseDocs = subDocs[upto].docsEnum = subDocsEnum;
+        subDocs[upto].slice = entry.subSlice;
+        upto++;
+      }
+    }
+
+    if (upto == 0) {
+      return null;
+    } else {
+      return docsEnum.reset(subDocs, upto);
+    }
+  }
+
+  @Override
+  public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+    final MultiDocsAndPositionsEnum docsAndPositionsEnum;
+    if (reuse != null) {
+      docsAndPositionsEnum = (MultiDocsAndPositionsEnum) reuse;
+    } else {
+      docsAndPositionsEnum = new MultiDocsAndPositionsEnum();
+    }
+    
+    final MultiBits multiSkipDocs;
+    if (skipDocs instanceof MultiBits) {
+      multiSkipDocs = (MultiBits) skipDocs;
+    } else {
+      multiSkipDocs = null;
+    }
+
+    int upto = 0;
+
+    for(int i=0;i<numTop;i++) {
+
+      final TermsEnumWithSlice entry = top[i];
+
+      final Bits b;
+
+      if (multiSkipDocs != null) {
+        // optimize for common case: requested skip docs is a
+        // congruent sub-slice of MultiBits: in this case, we
+        // just pull the skipDocs from the sub reader, rather
+        // than making the inefficient
+        // Slice(Multi(sub-readers)):
+        final Bits b2 = multiSkipDocs.getMatchingSub(top[i].subSlice);
+        if (b2 != null) {
+          b = b2;
+        } else {
+          // custom case: requested skip docs is foreign:
+          // must slice it on every access
+          b = new BitsSlice(skipDocs, top[i].subSlice);
+        }
+      } else {
+        // no deletions
+        b = null;
+      }
+
+      final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, entry.reusePostings);
+
+      if (subPostings != null) {
+        entry.reusePostings = subDocsAndPositions[upto].docsAndPositionsEnum = subPostings;
+        subDocsAndPositions[upto].slice = entry.subSlice;
+        upto++;
+      }
+    }
+
+    if (upto == 0) {
+      return null;
+    } else {
+      return docsAndPositionsEnum.reset(subDocsAndPositions, upto);
+    }
+  }
+
+  private final static class TermsEnumWithSlice {
+    private final ReaderUtil.Slice subSlice;
+    private TermsEnum terms;
+    private DocsEnum reuseDocs;
+    private DocsAndPositionsEnum reusePostings;
+    public BytesRef current;
+
+    public TermsEnumWithSlice(ReaderUtil.Slice subSlice) {
+      this.subSlice = subSlice;
+      assert subSlice.length >= 0: "length=" + subSlice.length;
+    }
+
+    public void reset(TermsEnum terms, BytesRef term) {
+      this.terms = terms;
+      current = term;
+      reuseDocs = null;
+      reusePostings = null;
+    }
+  }
+
+  private final static class TermMergeQueue extends PriorityQueue<TermsEnumWithSlice> {
+    BytesRef.Comparator termComp;
+    TermMergeQueue(int size) {
+      initialize(size);
+    }
+
+    @Override
+    protected final boolean lessThan(TermsEnumWithSlice termsA, TermsEnumWithSlice termsB) {
+      final int cmp = termComp.compare(termsA.current, termsB.current);
+      if (cmp != 0) {
+        return cmp < 0;
+      } else {
+        return termsA.subSlice.start < termsB.subSlice.start;
+      }
+    }
+  }
+}

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

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/ParallelReader.java Wed Feb 10 10:58:19 2010
@@ -147,7 +147,6 @@
     String currentField;
     IndexReader currentReader;
     Iterator<String> keys;
-    private final HashMap<String, IndexReader> readerFields = new HashMap<String, IndexReader>();
 
     ParallelFieldsEnum() {
       keys = fieldToReader.keySet().iterator();
@@ -168,7 +167,7 @@
     @Override
     public TermsEnum terms() throws IOException {
       assert currentReader != null;
-      Terms terms = currentReader.fields().terms(currentField);
+      Terms terms = MultiFields.getTerms(currentReader, currentField);
       if (terms != null) {
         return terms.iterator();
       } else {
@@ -182,7 +181,7 @@
     final HashMap<String,Terms> fields = new HashMap<String,Terms>();
 
     public void addField(String field, IndexReader r) throws IOException {
-      fields.put(field, r.fields().terms(field));
+      fields.put(field, MultiFields.getFields(r).terms(field));
     }
 
     @Override

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMerger.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMerger.java Wed Feb 10 10:58:19 2010
@@ -34,6 +34,8 @@
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.MultiBits;
 
 /**
  * The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
@@ -586,10 +588,22 @@
     // the new segment:
     codec = codecs.getWriter(state);
     
+    int docBase = 0;
+
+    final List<Fields> fields = new ArrayList<Fields>();
+    final List<IndexReader> subReaders = new ArrayList<IndexReader>();
+    final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
+
+    final int numReaders = readers.size();
+    for(int i=0;i<numReaders;i++) {
+      docBase = ReaderUtil.gatherSubFields(subReaders, fields, slices, readers.get(i), docBase);
+    }
+
+    // we may gather more readers than mergeState.readerCount
     mergeState = new MergeState();
-    mergeState.readers = readers;
+    mergeState.readers = subReaders;
+    mergeState.readerCount = subReaders.size();
     mergeState.fieldInfos = fieldInfos;
-    mergeState.readerCount = readers.size();
     mergeState.mergedDocCount = mergedDocs;
     
     // Remap docIDs
@@ -597,12 +611,23 @@
     mergeState.docMaps = new int[mergeState.readerCount][];
     mergeState.docBase = new int[mergeState.readerCount];
 
-    int docBase = 0;
+    docBase = 0;
+    int inputDocBase = 0;
+
+    final Bits[] subBits = new Bits[mergeState.readerCount];
+    final int[] starts = new int[mergeState.readerCount+1];
+
     for(int i=0;i<mergeState.readerCount;i++) {
-      final IndexReader reader = readers.get(i);
+
+      final IndexReader reader = subReaders.get(i);
+
+      starts[i] = inputDocBase;
+      subBits[i] = reader.getDeletedDocs();
+
       mergeState.delCounts[i] = reader.numDeletedDocs();
       mergeState.docBase[i] = docBase;
       docBase += reader.numDocs();
+      inputDocBase += reader.maxDoc();
       if (mergeState.delCounts[i] != 0) {
         int delCount = 0;
         Bits deletedDocs = reader.getDeletedDocs();
@@ -620,16 +645,16 @@
         assert delCount == mergeState.delCounts[i]: "reader delCount=" + mergeState.delCounts[i] + " vs recomputed delCount=" + delCount;
       }
     }
-
-    Fields[] fields = new Fields[mergeState.readerCount];
-    for(int i=0;i<mergeState.readerCount;i++) {
-      fields[i] = readers.get(i).fields();
-    }
+    starts[mergeState.readerCount] = inputDocBase;
 
     final FieldsConsumer consumer = codec.fieldsConsumer(state);
 
+    mergeState.multiDeletedDocs = new MultiBits(subBits, starts);
+    
     try {
-      consumer.merge(mergeState, fields);
+      consumer.merge(mergeState,
+                     new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
+                                     slices.toArray(ReaderUtil.Slice.EMPTY_ARRAY)));
     } finally {
       consumer.close();
     }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Terms.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Terms.java Wed Feb 10 10:58:19 2010
@@ -29,6 +29,8 @@
 
 public abstract class Terms {
 
+  public final static Terms[] EMPTY_ARRAY = new Terms[0];
+
   // Privately cache a TermsEnum per-thread for looking up
   // docFreq and getting a private DocsEnum
   private final CloseableThreadLocal<TermsEnum> threadEnums = new CloseableThreadLocal<TermsEnum>();

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsEnum.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsEnum.java Wed Feb 10 10:58:19 2010
@@ -57,6 +57,7 @@
    *  indicate whether exact term was found, a different
    *  term was found, or EOF was hit.  The target term may
    *  be befor or after the current term. */
+  // nocommit -- add boolean doCache?
   public abstract SeekStatus seek(BytesRef text) throws IOException;
 
   /** Seeks to the specified term by ordinal (position) as

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHashPerField.java Wed Feb 10 10:58:19 2010
@@ -412,7 +412,6 @@
           // can be inserted into the analyzer chain if
           // other behavior is wanted (pruning the term
           // to a prefix, throwing an exception, etc).
-
           if (docState.maxTermPrefix == null) {
             docState.maxTermPrefix = new String(tokenText, 0, 30);
           }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java Wed Feb 10 10:58:19 2010
@@ -21,7 +21,6 @@
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.util.PriorityQueue;
 
 import java.io.IOException;
 
@@ -40,98 +39,16 @@
   /** Called when we are done adding everything. */
   public abstract void close() throws IOException;
 
-  private final static class FieldMergeState {
-    String current;
-    FieldsEnum fieldsEnum;
-    int readerIndex;
-  }
-
-  // Used for merge-sorting by field
-  private final static class MergeQueue extends PriorityQueue<FieldMergeState> {
-    public MergeQueue(int size) {
-      initialize(size);
-    }
-
-    @Override
-    protected final boolean lessThan(FieldMergeState a, FieldMergeState b) {
-      final int cmp = a.current.compareTo(b.current);
-      if (cmp != 0) {
-        return cmp < 0;
-      } else {
-        // nocommit -- technically not required to break
-        // ties, since the terms merging will do so?
-        return a.readerIndex < b.readerIndex;
-      }
-    }
-  }
-
-  public void merge(MergeState mergeState, Fields[] fields) throws IOException {
-
-    MergeQueue queue = new MergeQueue(fields.length);
-
-    for(int i=0;i<fields.length;i++) {
-      FieldsEnum fieldsEnum = fields[i].iterator();
-      String field = fieldsEnum.next();
-      if (field != null) {
-        FieldMergeState state = new FieldMergeState();
-        state.current = field;
-        state.fieldsEnum = fieldsEnum;
-        state.readerIndex = i;
-        queue.add(state);
-      } else {
-        // no fields at all -- nothing to do
-      }
-    }
-
-    final FieldMergeState[] pending = new FieldMergeState[mergeState.readerCount];
-    final TermsConsumer.TermMergeState[] match = new TermsConsumer.TermMergeState[mergeState.readerCount];
-    for(int i=0;i<mergeState.readerCount;i++) {
-      match[i] = new TermsConsumer.TermMergeState();
-    }
-
-    // Merge sort by field name, calling terms.merge on all
-    // fields sharing same field name:
-    while(queue.size() != 0) {
-
-      int matchCount = 0;
-      int pendingCount = 0;
-
-      while(true) {
-        FieldMergeState state = pending[pendingCount++] = queue.pop();
-        TermsEnum termsEnum = state.fieldsEnum.terms();
-        if (termsEnum != null) {
-          match[matchCount].termsEnum = termsEnum;
-          match[matchCount].readerIndex = state.readerIndex;
-          matchCount++;
-        }
-        FieldMergeState top = queue.top();
-        if (top == null || top.current != pending[0].current) {
-          break;
-        }
-      }
-
-      if (matchCount > 0) {
-        // Merge one field
-        final String field = pending[0].current;
-        mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
-        mergeState.omitTermFreqAndPositions = mergeState.fieldInfo.omitTermFreqAndPositions;
-        final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
-        termsConsumer.merge(mergeState, match, matchCount);
-      }
-
-      // Put fields back into queue
-      for(int i=0;i<pendingCount;i++) {
-        FieldMergeState state = pending[i];
-        
-        state.current = state.fieldsEnum.next();
-        if (state.current != null) {
-          // More fields to merge
-          queue.add(state);
-        } else {
-          // Done
-        }
+  public void merge(MergeState mergeState, Fields fields) throws IOException {
+    FieldsEnum fieldsEnum = fields.iterator();
+    String field;
+    while((field = fieldsEnum.next()) != null) {
+      mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
+      final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
+      final TermsEnum termsEnum = fieldsEnum.terms();
+      if (termsEnum != null) {
+        termsConsumer.merge(mergeState, termsEnum);
       }
     }
-
   }
 }

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsAndPositionsEnum.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,121 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.MultiDocsAndPositionsEnum;
+import java.io.IOException;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments,
+ * remapping docIDs (this is used for segment merging).
+ *
+ * @lucene.experimental
+ */
+
+public final class MappingMultiDocsAndPositionsEnum extends DocsAndPositionsEnum {
+  private MultiDocsAndPositionsEnum.EnumWithSlice[] subs;
+  int numSubs;
+  int upto;
+  int[] currentMap;
+  DocsAndPositionsEnum current;
+  int currentBase;
+  int doc = -1;
+  private MergeState mergeState;
+
+  MappingMultiDocsAndPositionsEnum reset(MultiDocsAndPositionsEnum postingsEnum) throws IOException {
+    this.numSubs = postingsEnum.getNumSubs();
+    this.subs = postingsEnum.getSubs();
+    upto = -1;
+    current = null;
+    return this;
+  }
+
+  public void setMergeState(MergeState mergeState) {
+    this.mergeState = mergeState;
+  }
+
+  @Override
+  public int freq() {
+    return current.freq();
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    while(true) {
+      if (current == null) {
+        if (upto == numSubs-1) {
+          return this.doc = NO_MORE_DOCS;
+        } else {
+          upto++;
+          final int reader = subs[upto].slice.readerIndex;
+          current = subs[upto].docsAndPositionsEnum;
+          currentBase = mergeState.docBase[reader];
+          currentMap = mergeState.docMaps[reader];
+        }
+      }
+
+      int doc = current.nextDoc();
+      if (doc != NO_MORE_DOCS) {
+        if (currentMap != null) {
+          // compact deletions
+          doc = currentMap[doc];
+          if (doc == -1) {
+            continue;
+          }
+        }
+        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 BytesRef getPayload() throws IOException {
+    return current.getPayload();
+  }
+
+  @Override
+  public boolean hasPayload() {
+    return current.hasPayload();
+  }
+}
+

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

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MappingMultiDocsEnum.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,99 @@
+package org.apache.lucene.index.codecs;
+
+/**
+ * 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.index.DocsEnum;
+import org.apache.lucene.index.MultiDocsEnum;
+import java.io.IOException;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments,
+ * remapping docIDs (this is used for segment merging).
+ *
+ * @lucene.experimental
+ */
+
+public final class MappingMultiDocsEnum extends DocsEnum {
+  private MultiDocsEnum.EnumWithSlice[] subs;
+  int numSubs;
+  int upto;
+  int[] currentMap;
+  DocsEnum current;
+  int currentBase;
+  int doc = -1;
+  private MergeState mergeState;
+
+  MappingMultiDocsEnum reset(MultiDocsEnum docsEnum) throws IOException {
+    this.numSubs = docsEnum.getNumSubs();
+    this.subs = docsEnum.getSubs();
+    upto = -1;
+    current = null;
+    return this;
+  }
+
+  public void setMergeState(MergeState mergeState) {
+    this.mergeState = mergeState;
+  }
+
+  @Override
+  public int freq() {
+    return current.freq();
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    while(true) {
+      if (current == null) {
+        if (upto == numSubs-1) {
+          return this.doc = NO_MORE_DOCS;
+        } else {
+          upto++;
+          final int reader = subs[upto].slice.readerIndex;
+          current = subs[upto].docsEnum;
+          currentBase = mergeState.docBase[reader];
+          currentMap = mergeState.docMaps[reader];
+        }
+      }
+
+      int doc = current.nextDoc();
+      if (doc != NO_MORE_DOCS) {
+        if (currentMap != null) {
+          // compact deletions
+          doc = currentMap[doc];
+          if (doc == -1) {
+            continue;
+          }
+        }
+        return this.doc = currentBase + doc;
+      } else {
+        current = null;
+      }
+    }
+  }
+}
+

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

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MergeState.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MergeState.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MergeState.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/MergeState.java Wed Feb 10 10:58:19 2010
@@ -20,6 +20,7 @@
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.Bits;
 import java.util.List;
 
 /** Holds common state used during segment merging
@@ -34,9 +35,9 @@
   public int[] delCounts;                         // Deletion count per reader
   public int[] docBase;                           // New docID base per reader
   public int mergedDocCount;                      // Total # merged docs
+  public Bits multiDeletedDocs;
 
   // Updated per field;
   public FieldInfo fieldInfo;
-  public boolean omitTermFreqAndPositions;
 }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/PostingsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/PostingsConsumer.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/PostingsConsumer.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/PostingsConsumer.java Wed Feb 10 10:58:19 2010
@@ -31,12 +31,6 @@
 
   // nocommit
   public String desc;
-  /*
-  public boolean setDesc(String desc) {
-    this.desc = desc;
-    return true;
-  }
-  */
 
   // nocommit -- rename to startDoc?
   /** Adds a new doc in this term.  Return null if this
@@ -63,67 +57,42 @@
 
   /** Default merge impl: append documents, mapping around
    *  deletes */
-  public int merge(MergeState mergeState, PostingsMergeState[] toMerge, int count) throws IOException {
+  public int merge(MergeState mergeState, DocsEnum postings) throws IOException {
 
     int df = 0;
 
-    // Append docs in order:
-    for(int i=0;i<count;i++) {
-      final DocsEnum docsEnum = toMerge[i].docsEnum;
-      final int[] docMap = toMerge[i].docMap;
-      final int base = toMerge[i].docBase;
-
-      final DocsAndPositionsEnum postingsEnum;
-
-      if (!mergeState.omitTermFreqAndPositions) {
-        postingsEnum = (DocsAndPositionsEnum) docsEnum;
-      } else {
-        postingsEnum = null;
-      }
-
+    if (mergeState.fieldInfo.omitTermFreqAndPositions) {
       while(true) {
-        final int startDoc = docsEnum.nextDoc();
-        if (startDoc == DocsAndPositionsEnum.NO_MORE_DOCS) {
+        final int doc = postings.nextDoc();
+        if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) {
           break;
         }
+        addDoc(doc, postings.freq());
         df++;
-
-        int doc;
-        if (docMap != null) {
-          // map around deletions
-          doc = docMap[startDoc];
-          assert doc != -1: "docs enum returned deleted docID " + startDoc + " freq=" + docsEnum.freq() + " df=" + df + " de=" + docsEnum;
-        } else {
-          doc = startDoc;
+      }
+    } else {
+      final DocsAndPositionsEnum postingsEnum = (DocsAndPositionsEnum) postings;
+      while(true) {
+        final int doc = postingsEnum.nextDoc();
+        if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) {
+          break;
         }
-
-        doc += base;                              // convert to merged space
-        assert doc < mergeState.mergedDocCount: "doc=" + doc + " maxDoc=" + mergeState.mergedDocCount;
-
-        final int freq = docsEnum.freq();
-
+        final int freq = postingsEnum.freq();
         addDoc(doc, freq);
-
-        // nocommit -- omitTF should be "private", and this
-        // code (and FreqProxTermsWriter) should instead
-        // check if posConsumer is null?
-        if (!mergeState.omitTermFreqAndPositions) {
-          for(int j=0;j<freq;j++) {
-            final int position = postingsEnum.nextPosition();
-            final int payloadLength = postingsEnum.getPayloadLength();
-            final BytesRef payload;
-            if (payloadLength > 0) {
-              payload = postingsEnum.getPayload();
-            } else {
-              payload = null;
-            }
-            addPosition(position, payload);
+        for(int i=0;i<freq;i++) {
+          final int position = postingsEnum.nextPosition();
+          final int payloadLength = postingsEnum.getPayloadLength();
+          final BytesRef payload;
+          if (payloadLength > 0) {
+            payload = postingsEnum.getPayload();
+          } else {
+            payload = null;
           }
-          finishDoc();
+          addPosition(position, payload);
         }
+        df++;
       }
     }
-
     return df;
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/TermsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/TermsConsumer.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/TermsConsumer.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/TermsConsumer.java Wed Feb 10 10:58:19 2010
@@ -20,8 +20,9 @@
 import java.io.IOException;
 
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.index.MultiDocsEnum;
+import org.apache.lucene.index.MultiDocsAndPositionsEnum;
+
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -43,109 +44,46 @@
    *  before feeding to this API. */
   public abstract BytesRef.Comparator getComparator() throws IOException;
 
-  // For default merge impl
-  public static class TermMergeState {
-    BytesRef current;
-    TermsEnum termsEnum;
-    int readerIndex;
-  }
-
-  private final static class MergeQueue extends PriorityQueue<TermMergeState> {
-
-    final BytesRef.Comparator termComp;
-
-    public MergeQueue(int size, BytesRef.Comparator termComp) {
-      initialize(size);
-      this.termComp = termComp;
-    }
-
-    @Override
-    protected final boolean lessThan(TermMergeState a, TermMergeState b) {
-      final int cmp = termComp.compare(a.current, b.current);
-      if (cmp != 0) {
-        return cmp < 0;
-      } else {
-        return a.readerIndex < b.readerIndex;
-      }
-    }
-  }
-
-  private MergeQueue queue;
-  private PostingsConsumer.PostingsMergeState[] match;
-  private TermMergeState[] pending;
-
   /** Default merge impl */
-  public void merge(MergeState mergeState, TermMergeState[] termsStates, int count) throws IOException {
-
-    final BytesRef.Comparator termComp = getComparator();
+  private MappingMultiDocsEnum docsEnum = null;
+  private MappingMultiDocsAndPositionsEnum postingsEnum = null;
 
-    //System.out.println("merge terms field=" + mergeState.fieldInfo.name + " comp=" + termComp);
+  public void merge(MergeState mergeState, TermsEnum termsEnum) throws IOException {
 
-    if (queue == null) {
-      queue = new MergeQueue(mergeState.readerCount, termComp);
-      match = new PostingsConsumer.PostingsMergeState[mergeState.readerCount];
-      for(int i=0;i<mergeState.readerCount;i++) {
-        match[i] = new PostingsConsumer.PostingsMergeState();
-      }
-      pending = new TermMergeState[mergeState.readerCount];
-    } else if (!queue.termComp.equals(termComp)) {
-      queue = new MergeQueue(mergeState.readerCount, termComp);
-    }
+    BytesRef term;
 
-    // Init queue
-    for(int i=0;i<count;i++) {
-      TermMergeState state = termsStates[i];
-      state.current = state.termsEnum.next();
-      if (state.current != null) {
-        queue.add(state);
-      } else {
-        // no terms at all in this field
+    if (mergeState.fieldInfo.omitTermFreqAndPositions) {
+      if (docsEnum == null) {
+        docsEnum = new MappingMultiDocsEnum();
       }
-    }
-
-    while(queue.size() != 0) {
+      docsEnum.setMergeState(mergeState);
 
-      int matchCount = 0;
-      int pendingCount = 0;
+      MultiDocsEnum docsEnumIn = null;
 
-      while(true) {
-        TermMergeState state = pending[pendingCount++] = queue.pop();
-        
-        DocsEnum docsEnum = state.termsEnum.docsAndPositions(mergeState.readers.get(state.readerIndex).getDeletedDocs(), null);
-        if (docsEnum == null) {
-          docsEnum = state.termsEnum.docs(mergeState.readers.get(state.readerIndex).getDeletedDocs(), null);
-        }
-        if (docsEnum != null) {
-          match[matchCount].docsEnum = docsEnum;
-          match[matchCount].docMap = mergeState.docMaps[state.readerIndex];
-          match[matchCount].docBase = mergeState.docBase[state.readerIndex];
-          matchCount++;
-        }
-        TermMergeState top = queue.top();
-        if (top == null || !top.current.bytesEquals(pending[0].current)) {
-          break;
+      while((term = termsEnum.next()) != null) {
+        MultiDocsEnum docsEnumIn2 = (MultiDocsEnum) termsEnum.docs(mergeState.multiDeletedDocs, docsEnumIn);
+        if (docsEnumIn2 != null) {
+          docsEnumIn = docsEnumIn2;
+          docsEnum.reset(docsEnumIn);
+          final PostingsConsumer postingsConsumer = startTerm(term);
+          final int numDocs = postingsConsumer.merge(mergeState, docsEnum);
+          finishTerm(term, numDocs);
         }
       }
-
-      if (matchCount > 0) {
-        // Merge one term
-        final BytesRef term = pending[0].current;
-        //System.out.println("  merge term=" + term);
-        final PostingsConsumer postingsConsumer = startTerm(term);
-        final int numDocs = postingsConsumer.merge(mergeState, match, matchCount);
-        finishTerm(term, numDocs);
-      }
-
-      // Put terms back into queue
-      for(int i=0;i<pendingCount;i++) {
-        TermMergeState state = pending[i];
-        
-        state.current = state.termsEnum.next();
-        if (state.current != null) {
-          // More terms to merge
-          queue.add(state);
-        } else {
-          // Done
+    } else {
+      if (postingsEnum == null) {
+        postingsEnum = new MappingMultiDocsAndPositionsEnum();
+      }
+      postingsEnum.setMergeState(mergeState);
+      MultiDocsAndPositionsEnum postingsEnumIn = null;
+      while((term = termsEnum.next()) != null) {
+        MultiDocsAndPositionsEnum postingsEnumIn2 = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(mergeState.multiDeletedDocs, postingsEnumIn);
+        if (postingsEnumIn2 != null) {
+          postingsEnumIn = postingsEnumIn2;
+          postingsEnum.reset(postingsEnumIn);
+          final PostingsConsumer postingsConsumer = startTerm(term);
+          final int numDocs = postingsConsumer.merge(mergeState, postingsEnum);
+          finishTerm(term, numDocs);
         }
       }
     }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Wed Feb 10 10:58:19 2010
@@ -22,13 +22,11 @@
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.standard.TermState;
 import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Document;
 import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Position;
-import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -197,7 +195,8 @@
     }
   }
 
-  // nocommit -- not great that we can't always reuse
+  // TODO: we could actually reuse, by having TL that
+  // holds the last wrapped reuse, and vice-versa
   @Override
   public DocsEnum docs(FieldInfo field, TermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
     PulsingTermState termState = (PulsingTermState) _termState;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java Wed Feb 10 10:58:19 2010
@@ -24,6 +24,7 @@
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.util.ArrayUtil;
 
 import java.util.HashMap;
@@ -62,8 +63,10 @@
   private int indexDivisor;
   final private int indexInterval;
 
+  // Closed if indexLoaded is true:
   final private IndexInput in;
   private volatile boolean indexLoaded;
+
   private final BytesRef.Comparator termComp;
 
   final HashMap<FieldInfo,FieldIndexReader> fields = new HashMap<FieldInfo,FieldIndexReader>();
@@ -128,7 +131,6 @@
         }
       } else {
         this.in = in;
-        // nocommit -- we should close if index gets read on demand?
       }
     }
   }
@@ -139,17 +141,10 @@
    * usage of SegmentReader searching a tiny segment. */
   private final void trimByteBlock() {
     if (blockOffset == 0) {
-      // nocommit -- should not happen?  fields w/ no terms
-      // are not written by STDW.  hmmm it does
-      // happen... must explain why -- oh, could be only
-      // on exception; I added only calling this on
-      // success above
-      //assert false;
-      // nocommit -- hit AIOOBE here (blocks is length 0):
+      // There were no fields in this segment:
       if (blocks != null) {
         blocks[blockUpto] = null;
       }
-      //System.out.println("Simple terms index consumed no bytes! blockCount=" + blocks.length);
     } else {
       byte[] last = new byte[blockOffset];
       System.arraycopy(blocks[blockUpto], 0, last, 0, blockOffset);
@@ -157,7 +152,7 @@
     }
   }
 
-  // nocommit -- we can record precisely how many bytes are
+  // TODO: we can record precisely how many bytes are
   // required during indexing, save that into file, and be
   // precise when we allocate the blocks; we even don't need
   // to use blocks anymore (though my still want to, to
@@ -170,14 +165,18 @@
   int blockUpto;
   int blockOffset;
 
-  // nocommit -- is this big enough, given max allowed term
-  // size (measured in chars!!) ?
-  // nocommit -- or, we could allocate one block way to big,
-  // to accommodate such ridiculous terms
   private static final int BYTE_BLOCK_SHIFT = 15;
   private static final int BYTE_BLOCK_SIZE = 1 << BYTE_BLOCK_SHIFT;
   private static final int BYTE_BLOCK_MASK = BYTE_BLOCK_SIZE - 1;
 
+  static {
+    // Make sure DW can't ever write a term whose length
+    // cannot be encoded with short (because we use short[]
+    // to hold the length of each term).
+    assert IndexWriter.MAX_TERM_LENGTH < Short.MAX_VALUE;
+    assert BYTE_BLOCK_SIZE >= IndexWriter.MAX_TERM_LENGTH;
+  }
+
   private final class FieldIndexReader extends FieldReader {
 
     final private FieldInfo fieldInfo;
@@ -270,14 +269,13 @@
       // precise # bits
       final long[] blockPointer;
     
-      // Length of each term
-      // nocommit -- this is length in bytes; is short
-      // sufficient?  have to use negative space?
       // TODO: used packed ints here: we know max term
       // length; often its small
 
       // TODO: can we inline this w/ the bytes?  like
       // DW.  vast majority of terms only need 1 byte, not 2
+
+      // Length of each term
       final short[] termLength;
 
       final int numIndexTerms;
@@ -292,9 +290,6 @@
           // IndexWriter because a SegmentReader that at
           // first was opened for merging, is now being
           // opened to perform deletes or for an NRT reader
-
-          // nocommit -- how to allow apps to indexDivisor
-          // in this case?
           this.numIndexTerms = numIndexTerms;
         } else {
           this.numIndexTerms = 1+(numIndexTerms-1) / indexDivisor;
@@ -331,8 +326,6 @@
           final int suffix = clone.readVInt();
           final int thisTermLength = start + suffix;
 
-          // nocommit -- verify this is in fact guaranteed by
-          // DW -- we are talking bytes not chars here
           assert thisTermLength <= BYTE_BLOCK_SIZE;
 
           if (i%indexDivisor == 0) {
@@ -351,7 +344,6 @@
               blockOffset = 0;
             }
 
-            //System.out.println("blockUpto=" + blockUpto + " blocks.length=" + blocks.length);
             final byte[] block = blocks[blockUpto];
 
             // Copy old prefix
@@ -403,7 +395,6 @@
           }
         }
 
-        // nocommit: put in finally clause
         clone.close();
 
         assert upto == this.numIndexTerms;
@@ -501,8 +492,10 @@
       while(it.hasNext()) {
         it.next().loadTermsIndex();
       }
-      indexLoaded = true;
       trimByteBlock();
+
+      indexLoaded = true;
+      in.close();
     }
   }
 
@@ -526,7 +519,7 @@
 
   @Override
   public void close() throws IOException {
-    if (in != null) {
+    if (in != null && !indexLoaded) {
       in.close();
     }
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexWriter.java Wed Feb 10 10:58:19 2010
@@ -42,7 +42,6 @@
   private final FieldInfos fieldInfos; // unread
   private IndexOutput termsOut;
 
-  // nocommit
   final private String segment;
 
   public SimpleStandardTermsIndexWriter(SegmentWriteState state) throws IOException {
@@ -125,9 +124,6 @@
       out.writeLong(field.indexStart);
     }
     out.seek(Codec.headerSize(CODEC_NAME));
-    // nocommit -- why not simply write last 8 bytes of
-    // file?  hmm would require accurate filelength() in
-    // reader
     out.writeLong(dirStart);
     if (Codec.DEBUG) {
       System.out.println(" writeDirStart " + dirStart + " @ " + Codec.headerSize(CODEC_NAME));

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Wed Feb 10 10:58:19 2010
@@ -40,6 +40,8 @@
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
     StandardPostingsWriter docs = new StandardPostingsWriterImpl(state);
 
+    // nocommit -- how to gracefully upgrade to a new terms
+    // index impl?  could just make a new named codec...
     StandardTermsIndexWriter indexWriter;
     boolean success = false;
     try {
@@ -74,8 +76,6 @@
     StandardPostingsReader postings = new StandardPostingsReaderImpl(dir, si, readBufferSize);
     StandardTermsIndexReader indexReader;
 
-    // nocommit -- not clean that every codec must deal w/
-    // this... dup'd code
     boolean success = false;
     try {
       indexReader = new SimpleStandardTermsIndexReader(dir,

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Wed Feb 10 10:58:19 2010
@@ -27,11 +27,13 @@
 import org.apache.lucene.util.Bits;
 
 /** StandardTermsDictReader interacts with a single instance
- *  of this to manage creation of docs enum instances.  It
- *  provides an IndexInput (termsIn) where this class may
- *  read any previously stored data that it had written in
- *  its corresponding StandardDocsConsumer at indexing
+ *  of this to manage creation of {@link DocsEnum} and
+ *  {@link DocsAndPositionsEnum} instances.  It provides an
+ *  IndexInput (termsIn) where this class may read any
+ *  previously stored data that it had written in its
+ *  corresponding {@link StandardPostingsWriter} at indexing
  *  time. */
+
 public abstract class StandardPostingsReader implements Closeable {
 
   public abstract void init(IndexInput termsIn) throws IOException;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java Wed Feb 10 10:58:19 2010
@@ -359,6 +359,9 @@
           }
         }
 
+        // Useed only for assert:
+        final int startOrd;
+
         if (doSeek) {
 
           // As index to find biggest index term that's <=
@@ -379,15 +382,20 @@
           // scanning over an index term we'd have to
           // special case it:
           bytesReader.reset(indexResult.term);
-
+          
           state.ord = (int) indexResult.position-1;
           assert state.ord >= -1: "ord=" + state.ord;
 
+          startOrd = (int) indexResult.position;
+
           if (Codec.DEBUG) {
             Codec.debug("  set ord=" + state.ord);
           }
-        } else if (Codec.DEBUG) {
-          Codec.debug(": use scanning only (no seek)");
+        } else {
+          startOrd = -1;
+          if (Codec.DEBUG) {
+            Codec.debug(": use scanning only (no seek)");
+          }
         }
 
         // Now scan:
@@ -421,8 +429,12 @@
             return SeekStatus.NOT_FOUND;
           }
 
-          // nocommit -- put back assert that we don't cross
-          // another index term while scanning, here
+          // The purpose of the terms dict index is to seek
+          // the enum to the closest index term before the
+          // term we are looking for.  So, we should never
+          // cross another index term (besides the first
+          // one) while we are scanning:
+          assert state.ord == startOrd || !indexReader.isIndexTerm(state.ord, state.docFreq);
         }
 
         if (Codec.DEBUG) {
@@ -509,6 +521,12 @@
         // a "how many terms until next index entry" in each
         // index entry, but that'd require some tricky
         // lookahead work when writing the index
+
+        // nocommit -- this call to isIndexTerm is not
+        // right, when indexDivisor > 1?  ie, this will
+        // return false for entries that actually are index
+        // terms, and then the postings impl will read the
+        // wrong offset.  make a test...
         postingsReader.readTerm(in,
                                 fieldInfo, state,
                                 indexReader.isIndexTerm(1+state.ord, state.docFreq));

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictWriter.java Wed Feb 10 10:58:19 2010
@@ -61,7 +61,6 @@
   private final List<TermsConsumer> fields = new ArrayList<TermsConsumer>();
   private final BytesRef.Comparator termComp;
 
-  // nocommit
   private String segment;
 
   public StandardTermsDictWriter(StandardTermsIndexWriter indexWriter, SegmentWriteState state, StandardPostingsWriter postingsWriter, BytesRef.Comparator termComp) throws IOException {
@@ -107,8 +106,9 @@
   @Override
   public void close() throws IOException {
 
-    if (Codec.DEBUG)
+    if (Codec.DEBUG) {
       System.out.println("stdw.close seg=" + segment);
+    }
 
     try {
       final int fieldCount = fields.size();
@@ -142,14 +142,12 @@
     }
   }
 
-  long lastIndexPointer;
-
   class TermsWriter extends TermsConsumer {
-    final FieldInfo fieldInfo;
-    final StandardPostingsWriter postingsWriter;
-    final long termsStartPointer;
-    int numTerms;
-    final StandardTermsIndexWriter.FieldWriter fieldIndexWriter;
+    private final FieldInfo fieldInfo;
+    private final StandardPostingsWriter postingsWriter;
+    private final long termsStartPointer;
+    private int numTerms;
+    private final StandardTermsIndexWriter.FieldWriter fieldIndexWriter;
 
     TermsWriter(StandardTermsIndexWriter.FieldWriter fieldIndexWriter, FieldInfo fieldInfo, StandardPostingsWriter postingsWriter) {
       this.fieldInfo = fieldInfo;
@@ -158,7 +156,6 @@
       termWriter.reset();
       termsStartPointer = out.getFilePointer();
       postingsWriter.setField(fieldInfo);
-      lastIndexPointer = termsStartPointer;
       this.postingsWriter = postingsWriter;
 
       if (Codec.DEBUG) {
@@ -175,8 +172,8 @@
     public PostingsConsumer startTerm(BytesRef text) throws IOException {
       postingsWriter.startTerm();
       if (Codec.DEBUG) {
-        postingsWriter.desc = fieldInfo.name + ":" + text.toBytesString();
-        System.out.println("stdw.startTerm term=" + fieldInfo.name + ":" + text.toBytesString() + " seg=" + segment);
+        postingsWriter.desc = fieldInfo.name + ":" + text.toString();
+        System.out.println("stdw.startTerm term=" + fieldInfo.name + ":" + text.toString() + " seg=" + segment);
       }
       return postingsWriter;
     }
@@ -184,19 +181,16 @@
     @Override
     public void finishTerm(BytesRef text, int numDocs) throws IOException {
 
-      // mxx
       if (Codec.DEBUG) {
-        // nocommit     
-        Codec.debug("finishTerm seg=" + segment + " text=" + fieldInfo.name + ":" + text.toBytesString() + " numDocs=" + numDocs + " numTerms=" + numTerms);
+        Codec.debug("finishTerm seg=" + segment + " text=" + fieldInfo.name + ":" + text.toString() + " numDocs=" + numDocs + " numTerms=" + numTerms);
       }
 
       if (numDocs > 0) {
         final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, numDocs);
 
-        // mxx
         if (Codec.DEBUG) {
           Codec.debug("  tis.fp=" + out.getFilePointer() + " isIndexTerm?=" + isIndexTerm);
-          System.out.println("  term bytes=" + text.toBytesString());
+          System.out.println("  term bytes=" + text.toString());
         }
         termWriter.write(text);
         out.writeVInt(numDocs);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/AutomatonQuery.java Wed Feb 10 10:58:19 2010
@@ -23,6 +23,7 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.BasicAutomata;
@@ -72,12 +73,14 @@
   protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
     // matches nothing
     if (BasicOperations.isEmpty(automaton)) {
+      // nocommit -- should we just return null?  singleton?
       return new EmptyTermsEnum();
     }
     
     // matches all possible strings
     if (BasicOperations.isTotal(automaton)) {
-      final Terms terms = reader.fields().terms(getField());
+      final Terms terms = MultiFields.getTerms(reader, getField());
+      // nocommit -- should we just return null?  singleton?
       return (terms != null) ? terms.iterator() : new EmptyTermsEnum();
     }
     

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheImpl.java Wed Feb 10 10:58:19 2010
@@ -28,6 +28,7 @@
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Bits;
@@ -280,7 +281,7 @@
         return wrapper.getBytes(reader, field, FieldCache.DEFAULT_BYTE_PARSER);
       }
       final byte[] retArray = new byte[reader.maxDoc()];
-      Terms terms = reader.fields().terms(field);
+      Terms terms = MultiFields.getTerms(reader, field);
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator();
         final Bits delDocs = reader.getDeletedDocs();
@@ -334,7 +335,7 @@
         return wrapper.getShorts(reader, field, FieldCache.DEFAULT_SHORT_PARSER);
       }
       final short[] retArray = new short[reader.maxDoc()];
-      Terms terms = reader.fields().terms(field);
+      Terms terms = MultiFields.getTerms(reader, field);
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator();
         final Bits delDocs = reader.getDeletedDocs();
@@ -393,7 +394,7 @@
       }
       int[] retArray = null;
 
-      Terms terms = reader.fields().terms(field);
+      Terms terms = MultiFields.getTerms(reader, field);
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator();
         final Bits delDocs = reader.getDeletedDocs();
@@ -465,7 +466,7 @@
       }
       float[] retArray = null;
 
-      Terms terms = reader.fields().terms(field);
+      Terms terms = MultiFields.getTerms(reader, field);
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator();
         final Bits delDocs = reader.getDeletedDocs();
@@ -532,7 +533,8 @@
         }
       }
       long[] retArray = null;
-      Terms terms = reader.fields().terms(field);
+
+      Terms terms = MultiFields.getTerms(reader, field);
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator();
         final Bits delDocs = reader.getDeletedDocs();
@@ -601,7 +603,8 @@
         }
       }
       double[] retArray = null;
-      Terms terms = reader.fields().terms(field);
+
+      Terms terms = MultiFields.getTerms(reader, field);
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator();
         final Bits delDocs = reader.getDeletedDocs();
@@ -653,7 +656,7 @@
       String field = StringHelper.intern(entryKey.field);
       final String[] retArray = new String[reader.maxDoc()];
 
-      Terms terms = reader.fields().terms(field);
+      Terms terms = MultiFields.getTerms(reader, field);
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator();
         final Bits delDocs = reader.getDeletedDocs();
@@ -697,7 +700,7 @@
       String[] mterms = new String[reader.maxDoc()+1];
 
       //System.out.println("FC: getStringIndex field=" + field);
-      Terms terms = reader.fields().terms(field);
+      Terms terms = MultiFields.getTerms(reader, field);
 
       int t = 0;  // current term number
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredTermsEnum.java Wed Feb 10 10:58:19 2010
@@ -22,6 +22,7 @@
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.util.AttributeSource;
@@ -62,8 +63,12 @@
    * Creates a filtered {@link TermsEnum} for the given field name and reader.
    */
   public FilteredTermsEnum(final IndexReader reader, final String field) throws IOException {
-    final Terms terms = reader.fields().terms(field);
-    tenum = (terms != null) ? terms.iterator() : null;
+    final Terms terms = MultiFields.getTerms(reader, field);
+    if (terms != null) {
+      tenum = terms.iterator();
+    } else {
+      tenum = null;
+    }
   }
 
   /**

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PhraseQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PhraseQuery.java Wed Feb 10 10:58:19 2010
@@ -25,6 +25,7 @@
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.Explanation.IDFExplanation;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Bits;
@@ -157,11 +158,17 @@
       for (int i = 0; i < terms.size(); i++) {
         final Term t = terms.get(i);
         final BytesRef text = new BytesRef(t.text());
-        DocsAndPositionsEnum postingsEnum = reader.termPositionsEnum(delDocs,
-                                                                     t.field(),
-                                                                     text);
+        // NOTE: debateably, the caller should never pass in a
+        // multi reader...
+        DocsAndPositionsEnum postingsEnum = MultiFields.getTermPositionsEnum(reader,
+                                                                             delDocs,
+                                                                             t.field(),
+                                                                             text);
         if (postingsEnum == null) {
-          if (reader.termDocsEnum(delDocs, t.field(), text) != null) { 
+          if (MultiFields.getTermDocsEnum(reader,
+                                          delDocs,
+                                          t.field(),
+                                          text) != null) {
             // term does exist, but has no positions
             throw new IllegalStateException("field \"" + t.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run PhraseQuery (term=" + text + ")");
           } else {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/PrefixQuery.java Wed Feb 10 10:58:19 2010
@@ -23,6 +23,7 @@
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
 
 /** A Query that matches documents containing terms with a specified prefix. A PrefixQuery
@@ -51,7 +52,7 @@
   @Override  
   protected TermsEnum getTermsEnum(IndexReader reader) throws IOException {
     if (prefix.text().length() == 0) {
-      final Terms terms = reader.fields().terms(getField());
+      final Terms terms = MultiFields.getTerms(reader, getField());
       return (terms != null) ? terms.iterator() : new EmptyTermsEnum();
     }
     return new PrefixTermsEnum(reader, prefix);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermQuery.java Wed Feb 10 10:58:19 2010
@@ -22,6 +22,7 @@
 
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Explanation.IDFExplanation;
@@ -72,7 +73,9 @@
 
     @Override
     public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      DocsEnum docs = reader.termDocsEnum(reader.getDeletedDocs(), term.field(), new BytesRef(term.text()));
+      // NOTE: debateably, the caller should never pass in a
+      // multi reader...
+      DocsEnum docs = MultiFields.getTermDocsEnum(reader, reader.getDeletedDocs(), term.field(), new BytesRef(term.text()));
       if (docs == null) {
         return null;
       }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/TermRangeQuery.java Wed Feb 10 10:58:19 2010
@@ -23,6 +23,7 @@
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -140,7 +141,9 @@
       return new EmptyTermsEnum();
     }
     if ((lowerTerm == null || (collator == null && includeLower && "".equals(lowerTerm))) && upperTerm == null) {
-      final Terms terms = reader.fields().terms(field);
+      // NOTE: debateably, the caller should never pass in a
+      // multi reader...
+      final Terms terms = MultiFields.getTerms(reader, field);
       return (terms != null) ? terms.iterator() : new EmptyTermsEnum();
     }
     return new TermRangeTermsEnum(reader, field,

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Wed Feb 10 10:58:19 2010
@@ -19,6 +19,8 @@
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -40,7 +42,7 @@
   
   @Override
   public void extractTerms(Set<Term> terms) {
-	  terms.add(term);
+    terms.add(term);
   }
 
   @Override
@@ -81,10 +83,17 @@
 
   @Override
   public Spans getSpans(final IndexReader reader) throws IOException {
-    return new TermSpans(reader.termPositionsEnum(reader.getDeletedDocs(),
-                                                  term.field(),
-                                                  new BytesRef(term.text())),
-                         term);
+    // NOTE: debateably, the caller should never pass in a
+    // multi reader...
+    final DocsAndPositionsEnum postings = MultiFields.getTermPositionsEnum(reader,
+                                                                           reader.getDeletedDocs(),
+                                                                           term.field(),
+                                                                           new BytesRef(term.text()));
+
+    if (postings != null) {
+      return new TermSpans(postings, term);
+    } else {
+      return TermSpans.EMPTY_TERM_SPANS;
+    }
   }
-
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/Spans.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/Spans.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/Spans.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/Spans.java Wed Feb 10 10:58:19 2010
@@ -86,5 +86,4 @@
    * @return true if there is a payload available at this position that can be loaded
    */
   public abstract boolean isPayloadAvailable();
-
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/TermSpans.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/TermSpans.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/TermSpans.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/spans/TermSpans.java Wed Feb 10 10:58:19 2010
@@ -18,6 +18,7 @@
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
@@ -42,6 +43,12 @@
     doc = -1;
   }
 
+  // only for EmptyTermSpans (below)
+  TermSpans() {
+    term = null;
+    postings = null;
+  }
+
   @Override
   public boolean next() throws IOException {
     if (count == freq) {
@@ -119,4 +126,44 @@
   public DocsAndPositionsEnum getPostings() {
     return postings;
   }
+
+  private static final class EmptyTermSpans extends TermSpans {
+
+    @Override
+    public boolean next() {
+      return false;
+    }
+
+    @Override
+    public boolean skipTo(int target) {
+      return false;
+    }
+
+    @Override
+    public int doc() {
+      return DocIdSetIterator.NO_MORE_DOCS;
+    }
+    
+    @Override
+    public int start() {
+      return -1;
+    }
+
+    @Override
+    public int end() {
+      return -1;
+    }
+
+    @Override
+    public Collection<byte[]> getPayload() {
+      return null;
+    }
+
+    @Override
+    public boolean isPayloadAvailable() {
+      return false;
+    }
+  }
+
+  public static final TermSpans EMPTY_TERM_SPANS = new EmptyTermSpans();
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitVector.java Wed Feb 10 10:58:19 2010
@@ -110,6 +110,11 @@
     return size;
   }
 
+  @Override
+  public int length() {
+    return size;
+  }
+
   /** Returns the total number of one bits in this vector.  This is efficiently
     computed and cached, so that, if the vector is not changed, no
     recomputation is done for repeated calls. */

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Bits.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Bits.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Bits.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/Bits.java Wed Feb 10 10:58:19 2010
@@ -19,4 +19,5 @@
 
 public interface Bits {
   public boolean get(int index);
+  public int length();
 }

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitsSlice.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitsSlice.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitsSlice.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/BitsSlice.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,46 @@
+package org.apache.lucene.util;
+
+/**
+ * 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.
+ */
+
+/** Exposes a slice of an existing Bits as a new Bits. */
+
+public final class BitsSlice implements Bits {
+  private final Bits parent;
+  private final int start;
+  private final int length;
+
+  // start is inclusive; end is exclusive (length = end-start)
+  public BitsSlice(Bits parent, ReaderUtil.Slice slice) {
+    this.parent = parent;
+    this.start = slice.start;
+    this.length = slice.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);
+  }
+
+  public int length() {
+    return length;
+  }
+}

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

Added: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/MultiBits.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/MultiBits.java?rev=908447&view=auto
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/MultiBits.java (added)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/MultiBits.java Wed Feb 10 10:58:19 2010
@@ -0,0 +1,66 @@
+package org.apache.lucene.util;
+
+/**
+ * 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.
+ */
+
+/**
+ * Concatenates multiple Bits together, on every lookup.
+ *
+ * <p><b>NOTE</b>: This is very costly, as every lookup must
+ * do a binary search to locate the right sub-reader.
+ */
+public final class MultiBits implements Bits {
+  private final Bits[] subs;
+
+  // length is 1+subs.length (the last entry has the maxDoc):
+  private final int[] starts;
+
+  public MultiBits(Bits[] subs, int[] starts) {
+    this.subs = subs;
+    this.starts = starts;
+  }
+
+  private boolean checkLength(int reader, int doc) {
+    final int length = starts[1+reader]-starts[reader];
+    assert doc - starts[reader] < length: "doc=" + doc + " reader=" + reader + " starts[reader]=" + starts[reader] + " length=" + length;
+    return true;
+  }
+
+  public boolean get(int doc) {
+    final int reader = ReaderUtil.subIndex(doc, starts);
+    final Bits bits = subs[reader];
+    if (bits == null) {
+      return false;
+    } else {
+      assert checkLength(reader, doc);
+      return bits.get(doc-starts[reader]);
+    }
+  }
+
+  public Bits getMatchingSub(ReaderUtil.Slice slice) {
+    int reader = ReaderUtil.subIndex(slice.start, starts);
+    if (starts[reader] == slice.start && starts[1+reader] == slice.start+slice.length) {
+      return subs[reader];
+    } else {
+      return null;
+    }
+  }
+
+  public int length() {
+    return starts[starts.length-1];
+  }
+}

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

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/OpenBitSet.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/OpenBitSet.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/OpenBitSet.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/OpenBitSet.java Wed Feb 10 10:58:19 2010
@@ -134,6 +134,11 @@
       return capacity();
   }
 
+  @Override
+  public int length() {
+    return bits.length << 6;
+  }
+
   /** Returns true if there are no set bits */
   public boolean isEmpty() { return cardinality()==0; }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/ReaderUtil.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/ReaderUtil.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/util/ReaderUtil.java Wed Feb 10 10:58:19 2010
@@ -19,8 +19,10 @@
 
 import java.util.ArrayList;
 import java.util.List;
+import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Fields;
 
 /**
  * Common util methods for dealing with {@link IndexReader}s.
@@ -28,6 +30,23 @@
  */
 public class ReaderUtil {
 
+  public static class Slice {
+    public static final Slice[] EMPTY_ARRAY = new Slice[0];
+    public final int start;
+    public final int length;
+    public final int readerIndex;
+
+    public Slice(int start, int length, int readerIndex) {
+      this.start = start;
+      this.length = length;
+      this.readerIndex = readerIndex;
+    }
+
+    public String toString() {
+      return "slice start=" + start + " length=" + length;
+    }
+  }
+
   /**
    * Gathers sub-readers from reader into a List.
    * 
@@ -46,6 +65,25 @@
     }
   }
 
+  public static int gatherSubFields(List<IndexReader> readers, List<Fields> fields, List<Slice> slices, IndexReader reader, int base) throws IOException {
+    IndexReader[] subReaders = reader.getSequentialSubReaders();
+    if (subReaders == null) {
+      // Add the reader's fields
+      if (readers != null) {
+        readers.add(reader);
+      }
+      fields.add(reader.fields());
+      slices.add(new Slice(base, reader.maxDoc(), fields.size()-1));
+      base += reader.maxDoc();
+    } else {
+      for (int i = 0; i < subReaders.length; i++) {
+        base = gatherSubFields(readers, fields, slices, subReaders[i], base);
+      }
+    }
+
+    return base;
+  }
+
   /**
    * Returns sub IndexReader that contains the given document id.
    *    

Modified: lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestExternalCodecs.java?rev=908447&r1=908446&r2=908447&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/java/branches/flex_1458/src/test/org/apache/lucene/TestExternalCodecs.java Wed Feb 10 10:58:19 2010
@@ -846,7 +846,7 @@
 
     // Verify sort order matches what my comparator said:
     BytesRef lastBytesRef = null;
-    TermsEnum terms = r.fields().terms("id").iterator();
+    TermsEnum terms = MultiFields.getFields(r).terms("id").iterator();
     //System.out.println("id terms:");
     while(true) {
       BytesRef t = terms.next();