You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2010/12/20 11:53:33 UTC

svn commit: r1051056 [4/8] - in /lucene/dev/branches/docvalues: ./ lucene/ lucene/contrib/ant/src/java/org/apache/lucene/ant/ lucene/contrib/ant/src/test/org/apache/lucene/ant/ lucene/contrib/benchmark/src/test/org/apache/lucene/benchmark/quality/ luce...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon Dec 20 10:53:27 2010
@@ -32,7 +32,6 @@ import org.apache.lucene.index.codecs.Co
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.MergeState;
 import org.apache.lucene.index.codecs.FieldsConsumer;
-import org.apache.lucene.index.values.Type;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -58,18 +57,12 @@ final class SegmentMerger {
   private int termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
 
   private List<IndexReader> readers = new ArrayList<IndexReader>();
-  private FieldInfos fieldInfos;
+  private final FieldInfos fieldInfos;
   
   private int mergedDocs;
 
   private final CheckAbort checkAbort;
 
-  // Whether we should merge doc stores (stored fields and
-  // vectors files).  When all segments we are merging
-  // already share the same doc store files, we don't need
-  // to merge the doc stores.
-  private boolean mergeDocStores;
-
   /** Maximum number of contiguous documents to bulk-copy
       when merging stored fields */
   private final static int MAX_RAW_MERGE_DOCS = 4192;
@@ -80,10 +73,11 @@ final class SegmentMerger {
 
   private PayloadProcessorProvider payloadProcessorProvider;
   
-  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider) {
+  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
     this.payloadProcessorProvider = payloadProcessorProvider;
     directory = dir;
     this.codecs = codecs;
+    this.fieldInfos = fieldInfos;
     segment = name;
     if (merge != null) {
       checkAbort = new CheckAbort(merge, directory);
@@ -97,9 +91,9 @@ final class SegmentMerger {
     }
     this.termIndexInterval = termIndexInterval;
   }
-  
-  boolean hasProx() {
-    return fieldInfos.hasProx();
+
+  public FieldInfos fieldInfos() {
+    return fieldInfos;
   }
 
   /**
@@ -117,22 +111,6 @@ final class SegmentMerger {
    * @throws IOException if there is a low-level IO error
    */
   final int merge() throws CorruptIndexException, IOException {
-    return merge(true);
-  }
-
-  /**
-   * Merges the readers specified by the {@link #add} method
-   * into the directory passed to the constructor.
-   * @param mergeDocStores if false, we will not merge the
-   * stored fields nor vectors files
-   * @return The number of documents that were merged
-   * @throws CorruptIndexException if the index is corrupt
-   * @throws IOException if there is a low-level IO error
-   */
-  final int merge(boolean mergeDocStores) throws CorruptIndexException, IOException {
-
-    this.mergeDocStores = mergeDocStores;
-    
     // NOTE: it's important to add calls to
     // checkAbort.work(...) if you make any changes to this
     // method that will spend alot of time.  The frequency
@@ -144,7 +122,7 @@ final class SegmentMerger {
     mergeTerms();
     mergeNorms();
 
-    if (mergeDocStores && fieldInfos.hasVectors())
+    if (fieldInfos.hasVectors())
       mergeVectors();
     return mergedDocs;
   }
@@ -154,9 +132,7 @@ final class SegmentMerger {
 
     // Basic files
     for (String ext : IndexFileNames.COMPOUND_EXTENSIONS_NOT_CODEC) {
-      if (mergeDocStores || (!ext.equals(IndexFileNames.FIELDS_EXTENSION) &&
-                             !ext.equals(IndexFileNames.FIELDS_INDEX_EXTENSION)))
-        fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
+      fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
     }
     segmentWriteState.segmentCodecs.files(directory, info, fileSet);
     
@@ -171,7 +147,7 @@ final class SegmentMerger {
     }
 
     // Vector files
-    if (fieldInfos.hasVectors() && mergeDocStores) {
+    if (fieldInfos.hasVectors()) {
       for (String ext : IndexFileNames.VECTOR_EXTENSIONS) {
         fileSet.add(IndexFileNames.segmentFileName(segment, "", ext));
       }
@@ -204,13 +180,18 @@ final class SegmentMerger {
     for (String field : names) {
       fInfos.add(field, true, storeTermVectors,
           storePositionWithTermVector, storeOffsetWithTermVector, !reader
-              .hasNorms(field), storePayloads, omitTFAndPositions);
+              .hasNorms(field), storePayloads, omitTFAndPositions, null);
     }
   }
 
   private SegmentReader[] matchingSegmentReaders;
   private int[] rawDocLengths;
   private int[] rawDocLengths2;
+  private int matchedCount;
+
+  public int getMatchedSubReaderCount() {
+    return matchedCount;
+  }
 
   private void setMatchingSegmentReaders() {
     // If the i'th reader is a SegmentReader and has
@@ -235,6 +216,7 @@ final class SegmentMerger {
         }
         if (same) {
           matchingSegmentReaders[i] = segmentReader;
+          matchedCount++;
         }
       }
     }
@@ -250,18 +232,7 @@ final class SegmentMerger {
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  private final int mergeFields() throws CorruptIndexException, IOException {
-
-    if (!mergeDocStores) {
-      // When we are not merging by doc stores, their field
-      // name -> number mapping are the same.  So, we start
-      // with the fieldInfos of the last segment in this
-      // case, to keep that numbering.
-      final SegmentReader sr = (SegmentReader) readers.get(readers.size()-1);
-      fieldInfos = (FieldInfos) sr.core.fieldInfos.clone();
-    } else {
-      fieldInfos = new FieldInfos();// merge field names
-    }
+  private int mergeFields() throws CorruptIndexException, IOException {
 
     for (IndexReader reader : readers) {
       if (reader instanceof SegmentReader) {
@@ -269,19 +240,7 @@ final class SegmentMerger {
         FieldInfos readerFieldInfos = segmentReader.fieldInfos();
         int numReaderFieldInfos = readerFieldInfos.size();
         for (int j = 0; j < numReaderFieldInfos; j++) {
-          FieldInfo fi = readerFieldInfos.fieldInfo(j);
-          FieldInfo merged = fieldInfos.add(fi.name, fi.isIndexed, fi.storeTermVector,
-                                            fi.storePositionWithTermVector, fi.storeOffsetWithTermVector,
-                                            !reader.hasNorms(fi.name), fi.storePayloads,
-                                            fi.omitTermFreqAndPositions);
-          final Type fiIndexValues = fi.docValues;
-          final Type mergedDocValues = merged.docValues;
-          if (mergedDocValues == null) {
-            merged.setDocValues(fiIndexValues);
-          } else if (mergedDocValues != fiIndexValues) {
-            // TODO -- can we recover from this?
-            throw new IllegalStateException("cannot merge field " + fi.name + " indexValues changed from " + mergedDocValues + " to " + fiIndexValues);
-          }
+          fieldInfos.add(readerFieldInfos.fieldInfo(j));
         }
       } else {
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false);
@@ -302,54 +261,43 @@ final class SegmentMerger {
 
     setMatchingSegmentReaders();
 
-    if (mergeDocStores) {
-      // merge field values
-      final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
-
-      try {
-        int idx = 0;
-        for (IndexReader reader : readers) {
-          final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
-          FieldsReader matchingFieldsReader = null;
-          if (matchingSegmentReader != null) {
-            final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
-            if (fieldsReader != null) {
-              matchingFieldsReader = fieldsReader;
-            }
+    final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
+
+    try {
+      int idx = 0;
+      for (IndexReader reader : readers) {
+        final SegmentReader matchingSegmentReader = matchingSegmentReaders[idx++];
+        FieldsReader matchingFieldsReader = null;
+        if (matchingSegmentReader != null) {
+          final FieldsReader fieldsReader = matchingSegmentReader.getFieldsReader();
+          if (fieldsReader != null) {
+            matchingFieldsReader = fieldsReader;
           }
-          if (reader.hasDeletions()) {
-            docCount += copyFieldsWithDeletions(fieldsWriter,
-                                                reader, matchingFieldsReader);
-          } else {
-            docCount += copyFieldsNoDeletions(fieldsWriter,
+        }
+        if (reader.hasDeletions()) {
+          docCount += copyFieldsWithDeletions(fieldsWriter,
                                               reader, matchingFieldsReader);
-          }
+        } else {
+          docCount += copyFieldsNoDeletions(fieldsWriter,
+                                            reader, matchingFieldsReader);
         }
-      } finally {
-        fieldsWriter.close();
       }
+    } finally {
+      fieldsWriter.close();
+    }
 
-      final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
-      final long fdxFileLength = directory.fileLength(fileName);
-
-      if (4+((long) docCount)*8 != fdxFileLength)
-        // This is most likely a bug in Sun JRE 1.6.0_04/_05;
-        // we detect that the bug has struck, here, and
-        // throw an exception to prevent the corruption from
-        // entering the index.  See LUCENE-1282 for
-        // details.
-        throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
+    final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+    final long fdxFileLength = directory.fileLength(fileName);
 
-    } else {
-      // If we are skipping the doc stores, that means there
-      // are no deletions in any of these segments, so we
-      // just sum numDocs() of each segment to get total docCount
-      for (final IndexReader reader : readers) {
-        docCount += reader.numDocs();
-      }
-    }
+    if (4+((long) docCount)*8 != fdxFileLength)
+      // This is most likely a bug in Sun JRE 1.6.0_04/_05;
+      // we detect that the bug has struck, here, and
+      // throw an exception to prevent the corruption from
+      // entering the index.  See LUCENE-1282 for
+      // details.
+      throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
 
-    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, null, docCount, 0, termIndexInterval, codecInfo, new AtomicLong(0));
+    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, new AtomicLong(0));
     
     return docCount;
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon Dec 20 10:53:27 2010
@@ -248,7 +248,7 @@ public class SegmentReader extends Index
           throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + fieldsReaderOrig.size() + " but segmentInfo shows " + si.docCount);
         }
 
-        if (fieldInfos.hasVectors()) { // open term vector files only as needed
+        if (si.getHasVectors()) { // open term vector files only as needed
           termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, readBufferSize, si.getDocStoreOffset(), si.docCount);
         }
       }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Mon Dec 20 10:53:27 2010
@@ -32,9 +32,8 @@ public class SegmentWriteState {
   public final Directory directory;
   public final String segmentName;
   public final FieldInfos fieldInfos;
-  public final String docStoreSegmentName;
   public final int numDocs;
-  public int numDocsInStore;
+  public boolean hasVectors;
   public final Collection<String> flushedFiles;
   public final AtomicLong bytesUsed;
 
@@ -63,15 +62,12 @@ public class SegmentWriteState {
 
 
   public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
-                           String docStoreSegmentName, int numDocs,
-                           int numDocsInStore, int termIndexInterval, SegmentCodecs segmentCodecs, AtomicLong bytesUsed) {
+      int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, AtomicLong bytesUsed) {
     this.infoStream = infoStream;
     this.directory = directory;
     this.segmentName = segmentName;
     this.fieldInfos = fieldInfos;
-    this.docStoreSegmentName = docStoreSegmentName;
     this.numDocs = numDocs;
-    this.numDocsInStore = numDocsInStore;
     this.termIndexInterval = termIndexInterval;
     this.segmentCodecs = segmentCodecs;
     flushedFiles = new HashSet<String>();
@@ -87,9 +83,7 @@ public class SegmentWriteState {
     directory = state.directory;
     segmentName = state.segmentName;
     fieldInfos = state.fieldInfos;
-    docStoreSegmentName = state.docStoreSegmentName;
     numDocs = state.numDocs;
-    numDocsInStore = state.numDocsInStore;
     termIndexInterval = state.termIndexInterval;
     segmentCodecs = state.segmentCodecs;
     flushedFiles = state.flushedFiles;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon Dec 20 10:53:27 2010
@@ -29,7 +29,6 @@ final class StoredFieldsWriter {
   final DocumentsWriter docWriter;
   final FieldInfos fieldInfos;
   int lastDocID;
-  private String docStoreSegment;
 
   PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
@@ -44,60 +43,31 @@ final class StoredFieldsWriter {
   }
 
   synchronized public void flush(SegmentWriteState state) throws IOException {
-
-    if (state.numDocsInStore > 0) {
-      // It's possible that all documents seen in this segment
-      // hit non-aborting exceptions, in which case we will
-      // not have yet init'd the FieldsWriter:
-      initFieldsWriter();
-
-      // Fill fdx file to include any final docs that we
-      // skipped because they hit non-aborting exceptions
-      fill(state.numDocsInStore - docWriter.getDocStoreOffset());
-    }
-
-    if (fieldsWriter != null)
-      fieldsWriter.flush();
-  }
-
-  private synchronized void initFieldsWriter() throws IOException {
-    if (fieldsWriter == null) {
-      docStoreSegment = docWriter.getDocStoreSegment();
-      if (docStoreSegment != null) {
-        fieldsWriter = new FieldsWriter(docWriter.directory,
-                                        docStoreSegment,
-                                        fieldInfos);
-        docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
-        docWriter.addOpenFile(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
-        lastDocID = 0;
-      }
-    }
-  }
-
-  synchronized public void closeDocStore(SegmentWriteState state) throws IOException {
-    final int inc = state.numDocsInStore - lastDocID;
-    if (inc > 0) {
+    if (state.numDocs > lastDocID) {
       initFieldsWriter();
-      fill(state.numDocsInStore - docWriter.getDocStoreOffset());
+      fill(state.numDocs);
     }
 
     if (fieldsWriter != null) {
       fieldsWriter.close();
       fieldsWriter = null;
-      assert docStoreSegment != null;
-      assert state.docStoreSegmentName != null;
-      assert docStoreSegment.equals(state.docStoreSegmentName): "fieldsWriter wrote to segment=" + docStoreSegment + " vs SegmentWriteState segment=" + state.docStoreSegmentName;
       lastDocID = 0;
-      String fieldsName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_EXTENSION);
-      String fieldsIdxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
+
+      String fieldsName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_EXTENSION);
+      String fieldsIdxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
       state.flushedFiles.add(fieldsName);
       state.flushedFiles.add(fieldsIdxName);
 
-      docWriter.removeOpenFile(fieldsName);
-      docWriter.removeOpenFile(fieldsIdxName);
+      if (4 + ((long) state.numDocs) * 8 != state.directory.fileLength(fieldsIdxName)) {
+        throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
+      }
+    }
+  }
 
-      if (4+((long) state.numDocsInStore)*8 != state.directory.fileLength(fieldsIdxName))
-        throw new RuntimeException("after flush: fdx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(fieldsIdxName) + " length in bytes of " + fieldsIdxName + " file exists?=" + state.directory.fileExists(fieldsIdxName));
+  private synchronized void initFieldsWriter() throws IOException {
+    if (fieldsWriter == null) {
+      fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos);
+      lastDocID = 0;
     }
   }
 
@@ -114,16 +84,14 @@ final class StoredFieldsWriter {
         docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
-    } else
+    } else {
       return docFreeList[--freeCount];
+    }
   }
 
   synchronized void abort() {
     if (fieldsWriter != null) {
-      try {
-        fieldsWriter.close();
-      } catch (Throwable t) {
-      }
+      fieldsWriter.abort();
       fieldsWriter = null;
       lastDocID = 0;
     }
@@ -131,12 +99,9 @@ final class StoredFieldsWriter {
 
   /** Fills in any hole in the docIDs */
   void fill(int docID) throws IOException {
-    final int docStoreOffset = docWriter.getDocStoreOffset();
-
     // We must "catch up" for all docs before us
     // that had no stored fields:
-    final int end = docID+docStoreOffset;
-    while(lastDocID < end) {
+    while(lastDocID < docID) {
       fieldsWriter.skipDocument();
       lastDocID++;
     }
@@ -156,10 +121,6 @@ final class StoredFieldsWriter {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
-  public boolean freeRAM() {
-    return false;
-  }
-
   synchronized void free(PerDoc perDoc) {
     assert freeCount < docFreeList.length;
     assert 0 == perDoc.numStoredFields;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java Mon Dec 20 10:53:27 2010
@@ -77,38 +77,30 @@ class TermVectorsReader implements Clone
 
     try {
       String idxName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      if (d.fileExists(idxName)) {
-        tvx = d.openInput(idxName, readBufferSize);
-        format = checkValidFormat(tvx, idxName);
-        String fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-        tvd = d.openInput(fn, readBufferSize);
-        final int tvdFormat = checkValidFormat(tvd, fn);
-        fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
-        tvf = d.openInput(fn, readBufferSize);
-        final int tvfFormat = checkValidFormat(tvf, fn);
-
-        assert format == tvdFormat;
-        assert format == tvfFormat;
-
-        numTotalDocs = (int) (tvx.length() >> 4);
-
-        if (-1 == docStoreOffset) {
-          this.docStoreOffset = 0;
-          this.size = numTotalDocs;
-          assert size == 0 || numTotalDocs == size;
-        } else {
-          this.docStoreOffset = docStoreOffset;
-          this.size = size;
-          // Verify the file is long enough to hold all of our
-          // docs
-          assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
-        }
+      tvx = d.openInput(idxName, readBufferSize);
+      format = checkValidFormat(tvx, idxName);
+      String fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
+      tvd = d.openInput(fn, readBufferSize);
+      final int tvdFormat = checkValidFormat(tvd, fn);
+      fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
+      tvf = d.openInput(fn, readBufferSize);
+      final int tvfFormat = checkValidFormat(tvf, fn);
+
+      assert format == tvdFormat;
+      assert format == tvfFormat;
+
+      numTotalDocs = (int) (tvx.length() >> 4);
+
+      if (-1 == docStoreOffset) {
+        this.docStoreOffset = 0;
+        this.size = numTotalDocs;
+        assert size == 0 || numTotalDocs == size;
       } else {
-        // If all documents flushed in a segment had hit
-        // non-aborting exceptions, it's possible that
-        // FieldInfos.hasVectors returns true yet the term
-        // vector files don't exist.
-        format = 0;
+        this.docStoreOffset = docStoreOffset;
+        this.size = size;
+        // Verify the file is long enough to hold all of our
+        // docs
+        assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
       }
 
       this.fieldInfos = fieldInfos;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Mon Dec 20 10:53:27 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
@@ -30,13 +31,13 @@ import java.util.Map;
 final class TermVectorsTermsWriter extends TermsHashConsumer {
 
   final DocumentsWriter docWriter;
-  TermVectorsWriter termVectorsWriter;
   PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
   IndexOutput tvx;
   IndexOutput tvd;
   IndexOutput tvf;
   int lastDocID;
+  boolean hasVectors;
 
   public TermVectorsTermsWriter(DocumentsWriter docWriter) {
     this.docWriter = docWriter;
@@ -49,25 +50,29 @@ final class TermVectorsTermsWriter exten
 
   @Override
   synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
+    if (tvx != null) {
+      // At least one doc in this run had term vectors enabled
+      fill(state.numDocs);
+      tvx.close();
+      tvf.close();
+      tvd.close();
+      tvx = tvd = tvf = null;
+      assert state.segmentName != null;
+      String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
+      String fldName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
+      String docName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
 
-    // NOTE: it's possible that all documents seen in this segment
-    // hit non-aborting exceptions, in which case we will
-    // not have yet init'd the TermVectorsWriter.  This is
-    // actually OK (unlike in the stored fields case)
-    // because, although IieldInfos.hasVectors() will return
-    // true, the TermVectorsReader gracefully handles
-    // non-existence of the term vectors files.
+      if (4 + ((long) state.numDocs) * 16 != state.directory.fileLength(idxName)) {
+        throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
+      }
 
-    if (tvx != null) {
+      state.flushedFiles.add(idxName);
+      state.flushedFiles.add(fldName);
+      state.flushedFiles.add(docName);
 
-      if (state.numDocsInStore > 0)
-        // In case there are some final documents that we
-        // didn't see (because they hit a non-aborting exception):
-        fill(state.numDocsInStore - docWriter.getDocStoreOffset());
-
-      tvx.flush();
-      tvd.flush();
-      tvf.flush();
+      lastDocID = 0;
+      state.hasVectors = hasVectors;
+      hasVectors = false;
     }
 
     for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
@@ -82,35 +87,6 @@ final class TermVectorsTermsWriter exten
     }
   }
 
-  @Override
-  synchronized void closeDocStore(final SegmentWriteState state) throws IOException {
-    if (tvx != null) {
-      // At least one doc in this run had term vectors
-      // enabled
-      fill(state.numDocsInStore - docWriter.getDocStoreOffset());
-      tvx.close();
-      tvf.close();
-      tvd.close();
-      tvx = null;
-      assert state.docStoreSegmentName != null;
-      String idxName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      if (4+((long) state.numDocsInStore)*16 != state.directory.fileLength(idxName))
-        throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocsInStore + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName));
-
-      String fldName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
-      String docName = IndexFileNames.segmentFileName(state.docStoreSegmentName, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-      state.flushedFiles.add(idxName);
-      state.flushedFiles.add(fldName);
-      state.flushedFiles.add(docName);
-
-      docWriter.removeOpenFile(idxName);
-      docWriter.removeOpenFile(fldName);
-      docWriter.removeOpenFile(docName);
-
-      lastDocID = 0;
-    }    
-  }
-
   int allocCount;
 
   synchronized PerDoc getPerDoc() {
@@ -124,18 +100,17 @@ final class TermVectorsTermsWriter exten
         docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       }
       return new PerDoc();
-    } else
+    } else {
       return docFreeList[--freeCount];
+    }
   }
 
   /** Fills in no-term-vectors for all docs we haven't seen
    *  since the last doc that had term vectors. */
   void fill(int docID) throws IOException {
-    final int docStoreOffset = docWriter.getDocStoreOffset();
-    final int end = docID+docStoreOffset;
-    if (lastDocID < end) {
+    if (lastDocID < docID) {
       final long tvfPosition = tvf.getFilePointer();
-      while(lastDocID < end) {
+      while(lastDocID < docID) {
         tvx.writeLong(tvd.getFilePointer());
         tvd.writeVInt(0);
         tvx.writeLong(tvfPosition);
@@ -146,31 +121,20 @@ final class TermVectorsTermsWriter exten
 
   synchronized void initTermVectorsWriter() throws IOException {        
     if (tvx == null) {
-      
-      final String docStoreSegment = docWriter.getDocStoreSegment();
-
-      if (docStoreSegment == null)
-        return;
 
       // If we hit an exception while init'ing the term
       // vector output files, we must abort this segment
       // because those files will be in an unknown
       // state:
-      String idxName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      String docName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-      String fldName = IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
-      tvx = docWriter.directory.createOutput(idxName);
-      tvd = docWriter.directory.createOutput(docName);
-      tvf = docWriter.directory.createOutput(fldName);
+      hasVectors = true;
+      tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+      tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
+      tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
       
       tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
       tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
       tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
 
-      docWriter.addOpenFile(idxName);
-      docWriter.addOpenFile(fldName);
-      docWriter.addOpenFile(docName);
-
       lastDocID = 0;
     }
   }
@@ -188,8 +152,9 @@ final class TermVectorsTermsWriter exten
     tvx.writeLong(tvf.getFilePointer());
     tvd.writeVInt(perDoc.numVectorFields);
     if (perDoc.numVectorFields > 0) {
-      for(int i=0;i<perDoc.numVectorFields;i++)
+      for(int i=0;i<perDoc.numVectorFields;i++) {
         tvd.writeVInt(perDoc.fieldNumbers[i]);
+      }
       assert 0 == perDoc.fieldPointers[0];
       long lastPos = perDoc.fieldPointers[0];
       for(int i=1;i<perDoc.numVectorFields;i++) {
@@ -201,7 +166,7 @@ final class TermVectorsTermsWriter exten
       perDoc.numVectorFields = 0;
     }
 
-    assert lastDocID == perDoc.docID + docWriter.getDocStoreOffset();
+    assert lastDocID == perDoc.docID;
 
     lastDocID++;
 
@@ -210,35 +175,26 @@ final class TermVectorsTermsWriter exten
     assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
   }
 
-  public boolean freeRAM() {
-    // We don't hold any state beyond one doc, so we don't
-    // free persistent RAM here
-    return false;
-  }
-
   @Override
   public void abort() {
-    if (tvx != null) {
-      try {
-        tvx.close();
-      } catch (Throwable t) {
-      }
-      tvx = null;
-    }
-    if (tvd != null) {
-      try {
-        tvd.close();
-      } catch (Throwable t) {
-      }
-      tvd = null;
-    }
-    if (tvf != null) {
-      try {
-        tvf.close();
-      } catch (Throwable t) {
-      }
-      tvf = null;
+    hasVectors = false;
+    try {
+      IOUtils.closeSafely(tvx, tvd, tvf);
+    } catch (IOException ignored) {
+    }
+    try {
+      docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+    } catch (IOException ignored) {
+    }
+    try {
+      docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
+    } catch (IOException ignored) {
+    }
+    try {
+      docWriter.directory.deleteFile(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
+    } catch (IOException ignored) {
     }
+    tvx = tvd = tvf = null;
     lastDocID = 0;
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Mon Dec 20 10:53:27 2010
@@ -24,6 +24,7 @@ import org.apache.lucene.document.Fielda
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
 
 final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
 
@@ -298,7 +299,7 @@ final class TermVectorsTermsWriterPerFie
 
     @Override
     int bytesPerPosting() {
-      return super.bytesPerPosting() + 3 * DocumentsWriter.INT_NUM_BYTE;
+      return super.bytesPerPosting() + 3 * RamUsageEstimator.NUM_BYTES_INT;
     }
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon Dec 20 10:53:27 2010
@@ -70,13 +70,6 @@ final class TermsHash extends InvertedDo
   }
 
   @Override
-  synchronized void closeDocStore(SegmentWriteState state) throws IOException {
-    consumer.closeDocStore(state);
-    if (nextTermsHash != null)
-      nextTermsHash.closeDocStore(state);
-  }
-
-  @Override
   synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
     Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
     Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Mon Dec 20 10:53:27 2010
@@ -25,7 +25,6 @@ abstract class TermsHashConsumer {
   abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
   abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
   abstract void abort();
-  abstract void closeDocStore(SegmentWriteState state) throws IOException;
 
   FieldInfos fieldInfos;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Mon Dec 20 10:53:27 2010
@@ -40,7 +40,8 @@ public class DefaultSegmentInfosReader e
     IndexInput input = null;
     try {
       input = openInput(directory, segmentsFileName);
-      int format = input.readInt();
+      final int format = input.readInt();
+      infos.setFormat(format);
   
       // check that it is a format we can understand
       if (format > DefaultSegmentInfosWriter.FORMAT_MINIMUM)

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Mon Dec 20 10:53:27 2010
@@ -35,9 +35,12 @@ public class DefaultSegmentInfosWriter e
    *  diagnostics storage, and switches userData to Map */
   public static final int FORMAT_DIAGNOSTICS = -9;
 
+  /** Each segment records whether it has term vectors */
+  public static final int FORMAT_HAS_VECTORS = -10;
+
   /** Each segment records whether its postings are written
    *  in the new flex format */
-  public static final int FORMAT_4_0 = -10;
+  public static final int FORMAT_4_0 = -11;
 
   /** This must always point to the most recent file format.
    * whenever you add a new format, make it 1 smaller (negative version logic)! */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java Mon Dec 20 10:53:27 2010
@@ -20,16 +20,12 @@ package org.apache.lucene.index.codecs;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.FieldsEnum;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.codecs.docvalues.DocValuesConsumer;
 import org.apache.lucene.index.values.DocValues;
-import org.apache.lucene.index.values.Writer;
 
 import java.io.IOException;
 import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.List;
 
 /** Abstract API that consumes terms, doc, freq, prox and
  *  payloads postings.  Concrete implementations of this
@@ -47,8 +43,6 @@ public abstract class FieldsConsumer imp
   public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
     throw new UnsupportedOperationException("docvalues are not supported");
   }
-  
-
 
   /** Called when we are done adding everything. */
   public abstract void close() throws IOException;
@@ -67,9 +61,10 @@ public abstract class FieldsConsumer imp
       }
       if (mergeState.fieldInfo.hasDocValues()) {
         final DocValues docValues = fieldsEnum.docValues();
-        // TODO: is this assert values and if so when?
-//        assert docValues != null : "DocValues are null for " + mergeState.fieldInfo.getDocValues();
-        if(docValues == null) { // for now just continue
+        if(docValues == null) { 
+          /* It is actually possible that a fieldInfo has a values type but no values are actually available.
+           * this can happen if there are already segments without values around.
+           */
           continue; 
         }
         final DocValuesConsumer docValuesConsumer = addValuesField(mergeState.fieldInfo);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Mon Dec 20 10:53:27 2010
@@ -57,7 +57,7 @@ public class SimpleTextCodec extends Cod
   static final String POSTINGS_EXTENSION = "pst";
 
   static String getPostingsFileName(String segment, String id) {
-    return IndexFileNames.segmentFileName(segment, "", POSTINGS_EXTENSION);
+    return IndexFileNames.segmentFileName(segment, id, POSTINGS_EXTENSION);
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Mon Dec 20 10:53:27 2010
@@ -32,15 +32,16 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.automaton.fst.Builder;
+import org.apache.lucene.util.automaton.fst.BytesRefFSTEnum;
+import org.apache.lucene.util.automaton.fst.FST;
+import org.apache.lucene.util.automaton.fst.PositiveIntOutputs;
+import org.apache.lucene.util.automaton.fst.PairOutputs;
 
 import java.io.IOException;
 import java.util.Comparator;
 import java.util.Map;
-import java.util.Set;
 import java.util.HashMap;
-import java.util.TreeMap;
-import java.util.SortedMap;
-import java.util.Iterator;
 
 class SimpleTextFieldsReader extends FieldsProducer {
 
@@ -123,73 +124,39 @@ class SimpleTextFieldsReader extends Fie
   private class SimpleTextTermsEnum extends TermsEnum {
     private final IndexInput in;
     private final boolean omitTF;
-    private BytesRef current;
     private int docFreq;
     private long docsStart;
     private boolean ended;
-    private final TreeMap<BytesRef,TermData> allTerms;
-    private Iterator<Map.Entry<BytesRef,TermData>> iter;
+    private final BytesRefFSTEnum<PairOutputs.Pair<Long,Long>> fstEnum;
 
-    public SimpleTextTermsEnum(TreeMap<BytesRef,TermData> allTerms, boolean omitTF) throws IOException {
+    public SimpleTextTermsEnum(FST<PairOutputs.Pair<Long,Long>> fst, boolean omitTF) throws IOException {
       this.in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
-      this.allTerms = allTerms;
       this.omitTF = omitTF;
-      iter = allTerms.entrySet().iterator();
+      fstEnum = new BytesRefFSTEnum<PairOutputs.Pair<Long,Long>>(fst);
     }
 
     public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
-      
-      final SortedMap<BytesRef,TermData> tailMap = allTerms.tailMap(text);
 
-      if (tailMap.isEmpty()) {
-        current = null;
+      fstEnum.reset();
+      //System.out.println("seek to text=" + text.utf8ToString());
+      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,Long>> result = fstEnum.advance(text);
+      if (result == null) {
+        //System.out.println("  end");
         return SeekStatus.END;
       } else {
-        current = tailMap.firstKey();
-        final TermData td = tailMap.get(current);
-        docsStart = td.docsStart;
-        docFreq = td.docFreq;
-        iter = tailMap.entrySet().iterator();
-        assert iter.hasNext();
-        iter.next();
-        if (current.equals(text)) {
-          return SeekStatus.FOUND;
-        } else {
-          return SeekStatus.NOT_FOUND;
-        }
-      }
-
-      /*
-      if (current != null) {
-        final int cmp = current.compareTo(text);
-        if (cmp == 0) {
-          return SeekStatus.FOUND;
-        } else if (cmp > 0) {
-          ended = false;
-          in.seek(fieldStart);
-        }
-      } else {
-        ended = false;
-        in.seek(fieldStart);
-      }
+        //System.out.println("  got text=" + term.utf8ToString());
+        PairOutputs.Pair<Long,Long> pair = result.output;
+        docsStart = pair.output1;
+        docFreq = pair.output2.intValue();
 
-      // Naive!!  This just scans... would be better to do
-      // up-front scan to build in-RAM index
-      BytesRef b;
-      while((b = next()) != null) {
-        final int cmp = b.compareTo(text);
-        if (cmp == 0) {
-          ended = false;
+        if (result.input.equals(text)) {
+          //System.out.println("  match docsStart=" + docsStart);
           return SeekStatus.FOUND;
-        } else if (cmp > 0) {
-          ended = false;
+        } else {
+          //System.out.println("  not match docsStart=" + docsStart);
           return SeekStatus.NOT_FOUND;
         }
       }
-      current = null;
-      ended = true;
-      return SeekStatus.END;
-      */
     }
 
     @Override
@@ -199,56 +166,20 @@ class SimpleTextFieldsReader extends Fie
     @Override
     public BytesRef next() throws IOException {
       assert !ended;
-
-      if (iter.hasNext()) {
-        Map.Entry<BytesRef,TermData> ent = iter.next();
-        current = ent.getKey();
-        TermData td = ent.getValue();
-        docFreq = td.docFreq;
-        docsStart = td.docsStart;
-        return current;
+      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,Long>> result = fstEnum.next();
+      if (result != null) {
+        final PairOutputs.Pair<Long,Long> pair = result.output;
+        docsStart = pair.output1;
+        docFreq = pair.output2.intValue();
+        return result.input;
       } else {
-        current = null;
-        return null;
-      }
-
-      /*
-      readLine(in, scratch);
-      if (scratch.equals(END) || scratch.startsWith(FIELD)) {
-        ended = true;
-        current = null;
         return null;
-      } else {
-        assert scratch.startsWith(TERM): "got " + scratch.utf8ToString();
-        docsStart = in.getFilePointer();
-        final int len = scratch.length - TERM.length;
-        if (len > scratch2.length) {
-          scratch2.grow(len);
-        }
-        System.arraycopy(scratch.bytes, TERM.length, scratch2.bytes, 0, len);
-        scratch2.length = len;
-        current = scratch2;
-        docFreq = 0;
-        long lineStart = 0;
-        while(true) {
-          lineStart = in.getFilePointer();
-          readLine(in, scratch);
-          if (scratch.equals(END) || scratch.startsWith(FIELD) || scratch.startsWith(TERM)) {
-            break;
-          }
-          if (scratch.startsWith(DOC)) {
-            docFreq++;
-          }
-        }
-        in.seek(lineStart);
-        return current;
       }
-      */
     }
 
     @Override
     public BytesRef term() {
-      return current;
+      return fstEnum.current().input;
     }
 
     @Override
@@ -519,10 +450,7 @@ class SimpleTextFieldsReader extends Fie
     private final String field;
     private final long termsStart;
     private final boolean omitTF;
-
-    // NOTE: horribly, horribly RAM consuming, but then
-    // SimpleText should never be used in production
-    private final TreeMap<BytesRef,TermData> allTerms = new TreeMap<BytesRef,TermData>();
+    private FST<PairOutputs.Pair<Long,Long>> fst;
 
     private final BytesRef scratch = new BytesRef(10);
 
@@ -534,6 +462,8 @@ class SimpleTextFieldsReader extends Fie
     }
 
     private void loadTerms() throws IOException {
+      PositiveIntOutputs posIntOutputs = PositiveIntOutputs.getSingleton(false);
+      Builder<PairOutputs.Pair<Long,Long>> b = new Builder<PairOutputs.Pair<Long,Long>>(FST.INPUT_TYPE.BYTE1, 0, 0, true, new PairOutputs<Long,Long>(posIntOutputs, posIntOutputs));
       IndexInput in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
       in.seek(termsStart);
       final BytesRef lastTerm = new BytesRef(10);
@@ -543,16 +473,14 @@ class SimpleTextFieldsReader extends Fie
         readLine(in, scratch);
         if (scratch.equals(END) || scratch.startsWith(FIELD)) {
           if (lastDocsStart != -1) {
-            allTerms.put(new BytesRef(lastTerm),
-                         new TermData(lastDocsStart, docFreq));
+            b.add(lastTerm, new PairOutputs.Pair<Long,Long>(lastDocsStart, Long.valueOf(docFreq)));
           }
           break;
         } else if (scratch.startsWith(DOC)) {
           docFreq++;
         } else if (scratch.startsWith(TERM)) {
           if (lastDocsStart != -1) {
-            allTerms.put(new BytesRef(lastTerm),
-                         new TermData(lastDocsStart, docFreq));
+            b.add(lastTerm, new PairOutputs.Pair<Long,Long>(lastDocsStart, Long.valueOf(docFreq)));
           }
           lastDocsStart = in.getFilePointer();
           final int len = scratch.length - TERM.length;
@@ -564,11 +492,23 @@ class SimpleTextFieldsReader extends Fie
           docFreq = 0;
         }
       }
+      fst = b.finish();
+      /*
+      PrintStream ps = new PrintStream("out.dot");
+      fst.toDot(ps);
+      ps.close();
+      System.out.println("SAVED out.dot");
+      */
+      //System.out.println("FST " + fst.sizeInBytes());
     }
 
     @Override
     public TermsEnum iterator() throws IOException {
-      return new SimpleTextTermsEnum(allTerms, omitTF);
+      if (fst != null) {
+        return new SimpleTextTermsEnum(fst, omitTF);
+      } else {
+        return TermsEnum.EMPTY;
+      }
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Mon Dec 20 10:53:27 2010
@@ -82,16 +82,15 @@ public class StandardPostingsReader exte
     maxSkipLevels = termsIn.readInt();
   }
 
-  private static class DocTermState extends TermState {
+  // Must keep final because we do non-standard clone
+  private final static class DocTermState extends TermState {
     long freqOffset;
     long proxOffset;
     int skipOffset;
 
     public Object clone() {
-      DocTermState other = (DocTermState) super.clone();
-      other.freqOffset = freqOffset;
-      other.proxOffset = proxOffset;
-      other.skipOffset = skipOffset;
+      DocTermState other = new DocTermState();
+      other.copy(this);
       return other;
     }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java Mon Dec 20 10:53:27 2010
@@ -51,13 +51,14 @@ import org.apache.lucene.util.PagedBytes
  * </p>
  * @lucene.experimental
  */
-// TODO - add bulk copy where possible
 public final class Bytes {
+  //TODO - add bulk copy where possible
+  private Bytes() {  /* don't instantiate! */ }
 
-  // don't instantiate!
-  private Bytes() {
-  }
-
+  /**
+   *  
+   *
+   */
   public static enum Mode {
     STRAIGHT, DEREF, SORTED
   };
@@ -292,13 +293,13 @@ public final class Bytes {
         initIndexOut();
     }
 
-    protected void initDataOut() throws IOException {
+    private void initDataOut() throws IOException {
       datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
           DATA_EXTENSION));
       CodecUtil.writeHeader(datOut, codecName, version);
     }
 
-    protected void initIndexOut() throws IOException {
+    private void initIndexOut() throws IOException {
       idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
           INDEX_EXTENSION));
       CodecUtil.writeHeader(idxOut, codecName, version);
@@ -373,7 +374,6 @@ public final class Bytes {
       datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
           Writer.DATA_EXTENSION));
       version = CodecUtil.checkHeader(datIn, codecName, maxVersion, maxVersion);
-
       if (doIndex) {
         idxIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
             Writer.INDEX_EXTENSION));

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -51,20 +51,23 @@ class FixedDerefBytesImpl {
   static class Writer extends BytesWriterBase {
     private int size = -1;
     private int[] docToID;
-    private final BytesRefHash hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
-        new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+    private final BytesRefHash hash = new BytesRefHash(pool,
+        BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+            BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
 
-    public Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+    public Writer(Directory dir, String id, AtomicLong bytesUsed)
+        throws IOException {
       this(dir, id, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
           bytesUsed);
     }
 
     public Writer(Directory dir, String id, Allocator allocator,
         AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
           new ByteBlockPool(allocator), bytesUsed);
       docToID = new int[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash uses bytes too!
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash
+                                                            // uses bytes too!
     }
 
     @Override
@@ -73,7 +76,6 @@ class FixedDerefBytesImpl {
         return;
       if (size == -1) {
         size = bytes.length;
-        initDataOut();
         datOut.writeInt(size);
       } else if (bytes.length != size) {
         throw new IllegalArgumentException("expected bytes size=" + size
@@ -101,28 +103,31 @@ class FixedDerefBytesImpl {
     // some last docs that we didn't see
     @Override
     synchronized public void finish(int docCount) throws IOException {
-      if (datOut == null) // no added data
-        return;
-      initIndexOut();
-      final int count = 1 + hash.size();
-      idxOut.writeInt(count - 1);
-      // write index
-      final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
-          PackedInts.bitsRequired(count - 1));
-      final int limit = docCount > docToID.length ? docToID.length : docCount;
-      for (int i = 0; i < limit; i++) {
-        w.add(docToID[i]);
-      }
-      // fill up remaining doc with zeros
-      for (int i = limit; i < docCount; i++) {
-        w.add(0);
-      }
-      w.finish();
-      hash.close();
-      super.finish(docCount);
-      bytesUsed.addAndGet((-docToID.length)
-          * RamUsageEstimator.NUM_BYTES_INT);
-      docToID = null;
+      try {
+        if (size == -1) {
+          datOut.writeInt(size);
+        }
+        final int count = 1 + hash.size();
+        idxOut.writeInt(count - 1);
+        // write index
+        final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+            PackedInts.bitsRequired(count - 1));
+        final int limit = docCount > docToID.length ? docToID.length : docCount;
+        for (int i = 0; i < limit; i++) {
+          w.add(docToID[i]);
+        }
+        // fill up remaining doc with zeros
+        for (int i = limit; i < docCount; i++) {
+          w.add(0);
+        }
+        w.finish();
+      } finally {
+        hash.close();
+        super.finish(docCount);
+        bytesUsed
+            .addAndGet((-docToID.length) * RamUsageEstimator.NUM_BYTES_INT);
+        docToID = null;
+      }
     }
   }
 
@@ -131,11 +136,7 @@ class FixedDerefBytesImpl {
 
     Reader(Directory dir, String id, int maxDoc) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_START, true);
-      try {
-        size = datIn.readInt();
-      } catch (IOException e) {
-        throw e;
-      }
+      size = datIn.readInt();
     }
 
     @Override
@@ -184,8 +185,7 @@ class FixedDerefBytesImpl {
 
     @Override
     public DocValuesEnum getEnum(AttributeSource source) throws IOException {
-      return new DerefBytesEnum(source, cloneData(), cloneIndex(),
-          size);
+      return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
     }
 
     static class DerefBytesEnum extends DocValuesEnum {
@@ -202,8 +202,7 @@ class FixedDerefBytesImpl {
       }
 
       protected DerefBytesEnum(AttributeSource source, IndexInput datIn,
-          IndexInput idxIn, int size, Type enumType)
-          throws IOException {
+          IndexInput idxIn, int size, Type enumType) throws IOException {
         super(source, enumType);
         this.datIn = datIn;
         this.size = size;
@@ -215,11 +214,10 @@ class FixedDerefBytesImpl {
         bytesRef.offset = 0;
         valueCount = idx.size();
       }
-      
 
       protected void copyReferences(DocValuesEnum valuesEnum) {
         bytesRef = valuesEnum.bytesRef;
-        if(bytesRef.bytes.length < size) {
+        if (bytesRef.bytes.length < size) {
           bytesRef.grow(size);
         }
         bytesRef.length = size;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -57,18 +57,19 @@ class FixedSortedBytesImpl {
     private int[] docToEntry;
     private final Comparator<BytesRef> comp;
 
-    private final BytesRefHash hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
-        new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+    private final BytesRefHash hash = new BytesRefHash(pool,
+        BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+            BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
 
-    public Writer(Directory dir, String id, Comparator<BytesRef> comp, AtomicLong bytesUsed)
-        throws IOException {
+    public Writer(Directory dir, String id, Comparator<BytesRef> comp,
+        AtomicLong bytesUsed) throws IOException {
       this(dir, id, comp, new DirectAllocator(ByteBlockPool.BYTE_BLOCK_SIZE),
           bytesUsed);
     }
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Allocator allocator, AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false,
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, true, true,
           new ByteBlockPool(allocator), bytesUsed);
       docToEntry = new int[1];
       // docToEntry[0] = -1;
@@ -82,18 +83,15 @@ class FixedSortedBytesImpl {
         return; // default - skip it
       if (size == -1) {
         size = bytes.length;
-        initDataOut();
         datOut.writeInt(size);
       } else if (bytes.length != size) {
         throw new IllegalArgumentException("expected bytes size=" + size
             + " but got " + bytes.length);
       }
       if (docID >= docToEntry.length) {
-        int[] newArray = new int[ArrayUtil.oversize(1 + docID,
+        final int[] newArray = new int[ArrayUtil.oversize(1 + docID,
             RamUsageEstimator.NUM_BYTES_INT)];
         System.arraycopy(docToEntry, 0, newArray, 0, docToEntry.length);
-        // Arrays.fill(newArray, docToEntry.length, newArray.length, -1);
-
         bytesUsed.addAndGet((newArray.length - docToEntry.length)
             * RamUsageEstimator.NUM_BYTES_INT);
         docToEntry = newArray;
@@ -106,54 +104,56 @@ class FixedSortedBytesImpl {
     // some last docs that we didn't see
     @Override
     synchronized public void finish(int docCount) throws IOException {
-      if (datOut == null)// no data added
-        return;
-      initIndexOut();
-      final int[] sortedEntries = hash.sort(comp);
-      final int count = hash.size();
-      int[] address = new int[count];
-      // first dump bytes data, recording address as we go
-      for (int i = 0; i < count; i++) {
-        final int e = sortedEntries[i];
-        final BytesRef bytes = hash.get(e, new BytesRef());
-        assert bytes.length == size;
-        datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
-        address[e] = 1 + i;
-      }
-
-      idxOut.writeInt(count);
-
-      // next write index
-      PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount, PackedInts
-          .bitsRequired(count));
-      final int limit;
-      if (docCount > docToEntry.length) {
-        limit = docToEntry.length;
-      } else {
-        limit = docCount;
-      }
-      for (int i = 0; i < limit; i++) {
-        final int e = docToEntry[i];
-        if (e == 0) {
-          // null is encoded as zero
-          w.add(0);
+      try {
+        if (size == -1) {// no data added
+          datOut.writeInt(size);
+        }
+        final int[] sortedEntries = hash.sort(comp);
+        final int count = hash.size();
+        int[] address = new int[count];
+        // first dump bytes data, recording address as we go
+        for (int i = 0; i < count; i++) {
+          final int e = sortedEntries[i];
+          final BytesRef bytes = hash.get(e, new BytesRef());
+          assert bytes.length == size;
+          datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+          address[e] = 1 + i;
+        }
+
+        idxOut.writeInt(count);
+
+        // next write index
+        PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+            PackedInts.bitsRequired(count));
+        final int limit;
+        if (docCount > docToEntry.length) {
+          limit = docToEntry.length;
         } else {
-          assert e > 0 && e <= count : "index must  0 > && <= " + count
-              + " was: " + e;
-          w.add(address[e - 1]);
+          limit = docCount;
+        }
+        for (int i = 0; i < limit; i++) {
+          final int e = docToEntry[i];
+          if (e == 0) {
+            // null is encoded as zero
+            w.add(0);
+          } else {
+            assert e > 0 && e <= count : "index must  0 > && <= " + count
+                + " was: " + e;
+            w.add(address[e - 1]);
+          }
         }
-      }
 
-      for (int i = limit; i < docCount; i++) {
-        w.add(0);
+        for (int i = limit; i < docCount; i++) {
+          w.add(0);
+        }
+        w.finish();
+      } finally {
+        super.finish(docCount);
+        bytesUsed.addAndGet((-docToEntry.length)
+            * RamUsageEstimator.NUM_BYTES_INT);
+        docToEntry = null;
+        hash.close();
       }
-      w.finish();
-
-      super.finish(docCount);
-      bytesUsed.addAndGet((-docToEntry.length)
-          * RamUsageEstimator.NUM_BYTES_INT);
-      docToEntry = null;
-      hash.close();
     }
   }
 
@@ -187,9 +187,10 @@ class FixedSortedBytesImpl {
       private final int numValue;
       private final int size;
 
-      public Source(IndexInput datIn, IndexInput idxIn, int size, int numValues,
-          Comparator<BytesRef> comp) throws IOException {
-        super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), size*numValues );
+      public Source(IndexInput datIn, IndexInput idxIn, int size,
+          int numValues, Comparator<BytesRef> comp) throws IOException {
+        super(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), size
+            * numValues);
         this.size = size;
         this.numValue = numValues;
         index = PackedInts.getReader(idxIn);
@@ -209,9 +210,10 @@ class FixedSortedBytesImpl {
       public int getValueCount() {
         return numValue;
       }
+
       @Override
       protected BytesRef deref(int ord, BytesRef bytesRef) {
-        return data.fillSlice(bytesRef, (ord* size), size);
+        return data.fillSlice(bytesRef, (ord * size), size);
       }
 
       @Override
@@ -228,8 +230,7 @@ class FixedSortedBytesImpl {
     @Override
     public DocValuesEnum getEnum(AttributeSource source) throws IOException {
       // do unsorted
-      return new DerefBytesEnum(source, cloneData(), cloneIndex(),
-          size);
+      return new DerefBytesEnum(source, cloneData(), cloneIndex(), size);
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Mon Dec 20 10:53:27 2010
@@ -46,46 +46,49 @@ class FixedStraightBytesImpl {
     private byte[] oneRecord;
 
     protected Writer(Directory dir, String id) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, false, null, null);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, true, null, null);
     }
-    
+
     // TODO - impl bulk copy here!
 
     @Override
     synchronized public void add(int docID, BytesRef bytes) throws IOException {
       if (size == -1) {
         size = bytes.length;
-        initDataOut();
         datOut.writeInt(size);
         oneRecord = new byte[size];
       } else if (bytes.length != size) {
-        throw new IllegalArgumentException("expected bytes size=" + size + " but got " + bytes.length);
+        throw new IllegalArgumentException("expected bytes size=" + size
+            + " but got " + bytes.length);
       }
       fill(docID);
       assert bytes.bytes.length >= bytes.length;
       datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
     }
 
-    /* (non-Javadoc)
-     * @see org.apache.lucene.index.values.Writer#merge(org.apache.lucene.index.values.Writer.MergeState)
+    /*
+     * (non-Javadoc)
+     * 
+     * @see
+     * org.apache.lucene.index.values.Writer#merge(org.apache.lucene.index.values
+     * .Writer.MergeState)
      */
     @Override
     protected void merge(MergeState state) throws IOException {
-      if(state.bits == null && state.reader instanceof Reader){
+      if (state.bits == null && state.reader instanceof Reader) {
         Reader reader = (Reader) state.reader;
         final int maxDocs = reader.maxDoc;
-        if(maxDocs == 0)
+        if (maxDocs == 0)
           return;
-        if(size == -1) {
+        if (size == -1) {
           size = reader.size;
-          initDataOut();
           datOut.writeInt(size);
           oneRecord = new byte[size];
         }
-       fill(state.docBase);
-       // TODO should we add a transfer to API to each reader?
-       datOut.copyBytes(reader.cloneData(), size * maxDocs);
-       lastDocID += maxDocs-1;
+        fill(state.docBase);
+        // TODO should we add a transfer to API to each reader?
+        datOut.copyBytes(reader.cloneData(), size * maxDocs);
+        lastDocID += maxDocs - 1;
       } else
         super.merge(state);
     }
@@ -93,7 +96,7 @@ class FixedStraightBytesImpl {
     // Fills up to but not including this docID
     private void fill(int docID) throws IOException {
       assert size >= 0;
-      for(int i=lastDocID+1;i<docID;i++) {
+      for (int i = lastDocID + 1; i < docID; i++) {
         datOut.writeBytes(oneRecord, size);
       }
       lastDocID = docID;
@@ -101,24 +104,28 @@ class FixedStraightBytesImpl {
 
     @Override
     synchronized public void finish(int docCount) throws IOException {
-      if(datOut == null) // no data added
-        return;
-      fill(docCount);
-      super.finish(docCount);
+      try {
+        if (size == -1) {// no data added
+          datOut.writeInt(0);
+        } else {
+          fill(docCount);
+        }
+      } finally {
+        super.finish(docCount);
+      }
     }
 
     public long ramBytesUsed() {
       return 0;
     }
-    
+
   }
 
   public static class Reader extends BytesReaderBase {
     private final int size;
     private final int maxDoc;
 
-    Reader(Directory dir, String id, int maxDoc)
-      throws IOException {
+    Reader(Directory dir, String id, int maxDoc) throws IOException {
       super(dir, id, CODEC_NAME, VERSION_START, false);
       size = datIn.readInt();
       this.maxDoc = maxDoc;
@@ -138,15 +145,16 @@ class FixedStraightBytesImpl {
       private final int size;
       private final int maxDoc;
 
-      public Source(IndexInput datIn, IndexInput idxIn, int size, int maxDoc) throws IOException {
-        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size*maxDoc);
+      public Source(IndexInput datIn, IndexInput idxIn, int size, int maxDoc)
+          throws IOException {
+        super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc);
         this.size = size;
         this.missingValue.bytesValue = new BytesRef(size);
         this.maxDoc = maxDoc;
       }
-      
+
       @Override
-      public BytesRef getBytes(int docID, BytesRef bytesRef) { 
+      public BytesRef getBytes(int docID, BytesRef bytesRef) {
         return data.fillSlice(bytesRef, docID * size, size);
       }
 
@@ -170,7 +178,7 @@ class FixedStraightBytesImpl {
     public DocValuesEnum getEnum(AttributeSource source) throws IOException {
       return new FixedStraightBytesEnum(source, cloneData(), size, maxDoc);
     }
-    
+
     private static final class FixedStraightBytesEnum extends DocValuesEnum {
       private final IndexInput datIn;
       private final int size;
@@ -178,7 +186,8 @@ class FixedStraightBytesImpl {
       private int pos = -1;
       private final long fp;
 
-      public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn, int size, int maxDoc) throws IOException{
+      public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn,
+          int size, int maxDoc) throws IOException {
         super(source, Type.BYTES_FIXED_STRAIGHT);
         this.datIn = datIn;
         this.size = size;
@@ -188,45 +197,45 @@ class FixedStraightBytesImpl {
         bytesRef.offset = 0;
         fp = datIn.getFilePointer();
       }
-      
+
       protected void copyReferences(DocValuesEnum valuesEnum) {
         bytesRef = valuesEnum.bytesRef;
-        if(bytesRef.bytes.length < size) {
+        if (bytesRef.bytes.length < size) {
           bytesRef.grow(size);
         }
         bytesRef.length = size;
         bytesRef.offset = 0;
       }
-     
+
       public void close() throws IOException {
         datIn.close();
       }
-  
+
       @Override
       public int advance(int target) throws IOException {
-        if(target >= maxDoc){
+        if (target >= maxDoc || size == 0) {
           return pos = NO_MORE_DOCS;
         }
-        if((target-1) != pos) // pos inc == 1
+        if ((target - 1) != pos) // pos inc == 1
           datIn.seek(fp + target * size);
         datIn.readBytes(bytesRef.bytes, 0, size);
         return pos = target;
       }
-      
+
       @Override
       public int docID() {
         return pos;
       }
-      
+
       @Override
       public int nextDoc() throws IOException {
-        if(pos >= maxDoc){
+        if (pos >= maxDoc) {
           return pos = NO_MORE_DOCS;
         }
-        return advance(pos+1);
+        return advance(pos + 1);
       }
     }
-    
+
     @Override
     public Type type() {
       return Type.BYTES_FIXED_STRAIGHT;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java Mon Dec 20 10:53:27 2010
@@ -1,4 +1,5 @@
 package org.apache.lucene.index.values;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -33,6 +34,7 @@ import org.apache.lucene.util.FloatsRef;
 /**
  * Exposes writer/reader for floating point values. You can specify 4 (java
  * float) or 8 (java double) byte precision.
+ * 
  * @lucene.experimental
  */
 // TODO - add bulk copy where possible
@@ -45,8 +47,8 @@ public class Floats {
   private static final long LONG_DEFAULT = Double
       .doubleToRawLongBits(Double.NEGATIVE_INFINITY);
 
-  public static Writer getWriter(Directory dir, String id, int precisionBytes, AtomicLong bytesUsed)
-      throws IOException {
+  public static Writer getWriter(Directory dir, String id, int precisionBytes,
+      AtomicLong bytesUsed) throws IOException {
     if (precisionBytes != 4 && precisionBytes != 8) {
       throw new IllegalArgumentException("precisionBytes must be 4 or 8; got "
           + precisionBytes);
@@ -71,15 +73,16 @@ public class Floats {
     protected IndexOutput datOut;
     private final byte precision;
 
-    protected FloatsWriter(Directory dir, String id, int precision, AtomicLong bytesUsed)
-        throws IOException {
+    protected FloatsWriter(Directory dir, String id, int precision,
+        AtomicLong bytesUsed) throws IOException {
       super(bytesUsed);
       this.dir = dir;
       this.id = id;
       this.precision = (byte) precision;
+      initDatOut();
     }
 
-    protected void initDatOut() throws IOException {
+    private void initDatOut() throws IOException {
       datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
           Writer.DATA_EXTENSION));
       CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
@@ -98,7 +101,7 @@ public class Floats {
 
     @Override
     public void add(int docID, PerDocFieldValues docValues) throws IOException {
-        add(docID, docValues.getFloat());
+      add(docID, docValues.getFloat());
     }
 
     @Override
@@ -139,7 +142,8 @@ public class Floats {
   // Writes 4 bytes (float) per value
   static class Float4Writer extends FloatsWriter {
 
-    protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+    protected Float4Writer(Directory dir, String id, AtomicLong bytesUsed)
+        throws IOException {
       super(dir, id, 4, bytesUsed);
     }
 
@@ -148,9 +152,6 @@ public class Floats {
         throws IOException {
       assert docID > lastDocId : "docID: " + docID
           + " must be greater than the last added doc id: " + lastDocId;
-      if (datOut == null) {
-        initDatOut();
-      }
       if (docID - lastDocId > 1) {
         // fill with default values
         lastDocId += fillDefault(docID - lastDocId - 1);
@@ -162,13 +163,14 @@ public class Floats {
 
     @Override
     synchronized public void finish(int docCount) throws IOException {
-      if (datOut == null)
-        return; // no data added - don't create file!
-      if (docCount > lastDocId + 1)
-        for (int i = lastDocId; i < docCount; i++) {
-          datOut.writeInt(INT_DEFAULT); // default value
-        }
-      datOut.close();
+      try {
+        if (docCount > lastDocId + 1)
+          for (int i = lastDocId; i < docCount; i++) {
+            datOut.writeInt(INT_DEFAULT); // default value
+          }
+      } finally {
+        datOut.close();
+      }
     }
 
     @Override
@@ -183,7 +185,8 @@ public class Floats {
   // Writes 8 bytes (double) per value
   static class Float8Writer extends FloatsWriter {
 
-    protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+    protected Float8Writer(Directory dir, String id, AtomicLong bytesUsed)
+        throws IOException {
       super(dir, id, 8, bytesUsed);
     }
 
@@ -191,9 +194,6 @@ public class Floats {
     synchronized public void add(int docID, double v) throws IOException {
       assert docID > lastDocId : "docID: " + docID
           + " must be greater than the last added doc id: " + lastDocId;
-      if (datOut == null) {
-        initDatOut();
-      }
       if (docID - lastDocId > 1) {
         // fill with default values
         lastDocId += fillDefault(docID - lastDocId - 1);
@@ -205,13 +205,14 @@ public class Floats {
 
     @Override
     synchronized public void finish(int docCount) throws IOException {
-      if (datOut == null)
-        return; // no data added - don't create file!
-      if (docCount > lastDocId + 1)
-        for (int i = lastDocId; i < docCount; i++) {
-          datOut.writeLong(LONG_DEFAULT); // default value
-        }
-      datOut.close();
+      try {
+        if (docCount > lastDocId + 1)
+          for (int i = lastDocId; i < docCount; i++) {
+            datOut.writeLong(LONG_DEFAULT); // default value
+          }
+      } finally {
+        datOut.close();
+      }
     }
 
     @Override
@@ -288,7 +289,8 @@ public class Floats {
       }
 
       @Override
-      public DocValuesEnum getEnum(AttributeSource attrSource) throws IOException {
+      public DocValuesEnum getEnum(AttributeSource attrSource)
+          throws IOException {
         final MissingValue missing = getMissing();
         return new SourceEnum(attrSource, Type.SIMPLE_FLOAT_4BYTE, this, maxDoc) {
           @Override
@@ -327,7 +329,8 @@ public class Floats {
       }
 
       @Override
-      public DocValuesEnum getEnum(AttributeSource attrSource) throws IOException {
+      public DocValuesEnum getEnum(AttributeSource attrSource)
+          throws IOException {
         final MissingValue missing = getMissing();
         return new SourceEnum(attrSource, type(), this, maxDoc) {
           @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java?rev=1051056&r1=1051055&r2=1051056&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java Mon Dec 20 10:53:27 2010
@@ -32,7 +32,8 @@ import org.apache.lucene.util.OpenBitSet
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.PackedInts;
 
-/** Stores ints packed with fixed-bit precision.
+/**
+ * Stores ints packed with fixed-bit precision.
  * 
  * @lucene.experimental
  * */
@@ -51,17 +52,21 @@ class PackedIntsImpl {
     private long minValue;
     private long maxValue;
     private boolean started;
-    private final Directory dir;
     private final String id;
     private final OpenBitSet defaultValues = new OpenBitSet(1);
     private int lastDocId = -1;
+    private IndexOutput datOut;
 
-    protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed) throws IOException {
+    protected IntsWriter(Directory dir, String id, AtomicLong bytesUsed)
+        throws IOException {
       super(bytesUsed);
-      this.dir = dir;
       this.id = id;
       docToValue = new long[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG); // TODO the bitset needs memory too
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG); // TODO the bitset
+                                                             // needs memory too
+      datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+          DATA_EXTENSION));
+      CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
     }
 
     @Override
@@ -81,50 +86,55 @@ class PackedIntsImpl {
       lastDocId = docID;
 
       if (docID >= docToValue.length) {
-        final long len = docToValue.length ;
+        final long len = docToValue.length;
         docToValue = ArrayUtil.grow(docToValue, 1 + docID);
         defaultValues.ensureCapacity(docToValue.length);
-        bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG * ((docToValue.length) - len));
+        bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_LONG
+            * ((docToValue.length) - len));
       }
       docToValue[docID] = v;
     }
 
     @Override
     public synchronized void finish(int docCount) throws IOException {
-      if (!started)
-        return;
-      final IndexOutput datOut = dir.createOutput(IndexFileNames
-          .segmentFileName(id, "", DATA_EXTENSION));
-      CodecUtil.writeHeader(datOut, CODEC_NAME, VERSION_CURRENT);
+      try {
+        if (!started) {
+          minValue = maxValue = 0;
+        }
+        // TODO -- long can't work right since it's signed
+        datOut.writeLong(minValue);
+        // write a default value to recognize docs without a value for that
+        // field
+        final long defaultValue = ++maxValue - minValue;
+        datOut.writeLong(defaultValue);
+        PackedInts.Writer w = PackedInts.getWriter(datOut, docCount,
+            PackedInts.bitsRequired(maxValue - minValue));
+        final int firstDoc = defaultValues.nextSetBit(0);
+        lastDocId++;
+        if(firstDoc != -1) { 
+          for (int i = 0; i < firstDoc; i++) {
+            w.add(defaultValue); // fill with defaults until first bit set
+          }
 
-      // TODO -- long can't work right since it's signed
-      datOut.writeLong(minValue);
-      // write a default value to recognize docs without a value for that field
-      final long defaultValue = ++maxValue - minValue;
-      datOut.writeLong(defaultValue);
-      PackedInts.Writer w = PackedInts.getWriter(datOut, docCount, PackedInts
-          .bitsRequired(maxValue - minValue));
-      final int firstDoc = defaultValues.nextSetBit(0);
-      assert firstDoc >= 0; // we have at lest one value!
-      for (int i = 0; i < firstDoc; i++) {
-        w.add(defaultValue); // fill with defaults until first bit set
-      }
-      lastDocId++;
-      for (int i = firstDoc; i < lastDocId;) {
-        w.add(docToValue[i] - minValue);
-        final int nextValue = defaultValues.nextSetBit(i);
-        for (i++; i < nextValue; i++) {
-          w.add(defaultValue); // fill all gaps
+          for (int i = firstDoc; i < lastDocId;) {
+            w.add(docToValue[i] - minValue);
+            final int nextValue = defaultValues.nextSetBit(i);
+            for (i++; i < nextValue; i++) {
+              w.add(defaultValue); // fill all gaps
+            }
+          }
         }
+        for (int i = lastDocId; i < docCount; i++) {
+          w.add(defaultValue);
+        }
+        w.finish();
+      } finally {
+        datOut.close();
+        bytesUsed
+            .addAndGet(-(RamUsageEstimator.NUM_BYTES_LONG * docToValue.length));
+        docToValue = null;
       }
-      for (int i = lastDocId; i < docCount; i++) {
-        w.add(defaultValue);
-      }
-      w.finish();
-      datOut.close();
-      bytesUsed.addAndGet(-(RamUsageEstimator.NUM_BYTES_LONG * docToValue.length ));
-      docToValue = null;
-      
+
     }
 
     @Override
@@ -139,7 +149,7 @@ class PackedIntsImpl {
 
     @Override
     public void add(int docID, PerDocFieldValues docValues) throws IOException {
-        add(docID, docValues.getInt());
+      add(docID, docValues.getInt());
     }
 
     @Override
@@ -192,7 +202,8 @@ class PackedIntsImpl {
       }
 
       @Override
-      public DocValuesEnum getEnum(AttributeSource attrSource) throws IOException {
+      public DocValuesEnum getEnum(AttributeSource attrSource)
+          throws IOException {
         final MissingValue missing = getMissing();
         return new SourceEnum(attrSource, type(), this, values.size()) {
           @Override