lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sim...@apache.org
Subject svn commit: r1343568 [4/10] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ lucene/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/standard/std31/ lucene/analysis/common/src/java/org/apache/lucene/analysis/stand...
Date Tue, 29 May 2012 07:22:18 GMT
Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java Tue May 29 07:22:08 2012
@@ -18,14 +18,15 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
-import java.util.Set;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.lucene.codecs.FieldInfosReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.store.Directory;
@@ -51,10 +52,6 @@ public class SimpleTextFieldInfosReader 
     IndexInput input = directory.openInput(fileName, iocontext);
     BytesRef scratch = new BytesRef();
     
-    boolean hasVectors = false;
-    boolean hasFreq = false;
-    boolean hasProx = false;
-    
     try {
       
       SimpleTextUtil.readLine(input, scratch);
@@ -97,25 +94,35 @@ public class SimpleTextFieldInfosReader 
         String dvType = readString(DOCVALUES.length, scratch);
         final DocValues.Type docValuesType = docValuesType(dvType);
         
-        
-        
         SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch, INDEXOPTIONS);
         IndexOptions indexOptions = IndexOptions.valueOf(readString(INDEXOPTIONS.length, scratch));
+      
+        SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch, NUM_ATTS);
+        int numAtts = Integer.parseInt(readString(NUM_ATTS.length, scratch));
+        Map<String,String> atts = new HashMap<String,String>();
+
+        for (int j = 0; j < numAtts; j++) {
+          SimpleTextUtil.readLine(input, scratch);
+          assert StringHelper.startsWith(scratch, ATT_KEY);
+          String key = readString(ATT_KEY.length, scratch);
+        
+          SimpleTextUtil.readLine(input, scratch);
+          assert StringHelper.startsWith(scratch, ATT_VALUE);
+          String value = readString(ATT_VALUE.length, scratch);
+          atts.put(key, value);
+        }
 
-        hasVectors |= storeTermVector;
-        hasProx |= isIndexed && indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
-        hasFreq |= isIndexed && indexOptions != IndexOptions.DOCS_ONLY;
-        
         infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
-          omitNorms, storePayloads, indexOptions, docValuesType, normsType);
+          omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(atts));
       }
 
       if (input.getFilePointer() != input.length()) {
         throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
       }
       
-      return new FieldInfos(infos, hasFreq, hasProx, hasVectors);
+      return new FieldInfos(infos);
     } finally {
       input.close();
     }
@@ -132,8 +139,4 @@ public class SimpleTextFieldInfosReader 
   private String readString(int offset, BytesRef scratch) {
     return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
   }
-  
-  public static void files(SegmentInfo info, Set<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(info.name, "", FIELD_INFOS_EXTENSION));
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java Tue May 29 07:22:08 2012
@@ -17,6 +17,7 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 import java.io.IOException;
+import java.util.Map;
 
 import org.apache.lucene.codecs.FieldInfosWriter;
 import org.apache.lucene.index.DocValues;
@@ -52,6 +53,9 @@ public class SimpleTextFieldInfosWriter 
   static final BytesRef NORMS_TYPE      =  new BytesRef("  norms type ");
   static final BytesRef DOCVALUES       =  new BytesRef("  doc values ");
   static final BytesRef INDEXOPTIONS    =  new BytesRef("  index options ");
+  static final BytesRef NUM_ATTS        =  new BytesRef("  attributes ");
+  final static BytesRef ATT_KEY         =  new BytesRef("    key ");
+  final static BytesRef ATT_VALUE       =  new BytesRef("    value ");
   
   @Override
   public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
@@ -64,7 +68,7 @@ public class SimpleTextFieldInfosWriter 
       SimpleTextUtil.writeNewline(out);
       
       for (FieldInfo fi : infos) {
-        assert fi.indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.storePayloads;
+        assert fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
 
         SimpleTextUtil.write(out, NAME);
         SimpleTextUtil.write(out, fi.name, scratch);
@@ -75,19 +79,19 @@ public class SimpleTextFieldInfosWriter 
         SimpleTextUtil.writeNewline(out);
         
         SimpleTextUtil.write(out, ISINDEXED);
-        SimpleTextUtil.write(out, Boolean.toString(fi.isIndexed), scratch);
+        SimpleTextUtil.write(out, Boolean.toString(fi.isIndexed()), scratch);
         SimpleTextUtil.writeNewline(out);
         
         SimpleTextUtil.write(out, STORETV);
-        SimpleTextUtil.write(out, Boolean.toString(fi.storeTermVector), scratch);
+        SimpleTextUtil.write(out, Boolean.toString(fi.hasVectors()), scratch);
         SimpleTextUtil.writeNewline(out);
         
         SimpleTextUtil.write(out, PAYLOADS);
-        SimpleTextUtil.write(out, Boolean.toString(fi.storePayloads), scratch);
+        SimpleTextUtil.write(out, Boolean.toString(fi.hasPayloads()), scratch);
         SimpleTextUtil.writeNewline(out);
                
         SimpleTextUtil.write(out, NORMS);
-        SimpleTextUtil.write(out, Boolean.toString(!fi.omitNorms), scratch);
+        SimpleTextUtil.write(out, Boolean.toString(!fi.omitsNorms()), scratch);
         SimpleTextUtil.writeNewline(out);
         
         SimpleTextUtil.write(out, NORMS_TYPE);
@@ -99,8 +103,26 @@ public class SimpleTextFieldInfosWriter 
         SimpleTextUtil.writeNewline(out);
         
         SimpleTextUtil.write(out, INDEXOPTIONS);
-        SimpleTextUtil.write(out, fi.indexOptions.toString(), scratch);
+        SimpleTextUtil.write(out, fi.getIndexOptions().toString(), scratch);
         SimpleTextUtil.writeNewline(out);
+        
+        Map<String,String> atts = fi.attributes();
+        int numAtts = atts == null ? 0 : atts.size();
+        SimpleTextUtil.write(out, NUM_ATTS);
+        SimpleTextUtil.write(out, Integer.toString(numAtts), scratch);
+        SimpleTextUtil.writeNewline(out);
+      
+        if (numAtts > 0) {
+          for (Map.Entry<String,String> entry : atts.entrySet()) {
+            SimpleTextUtil.write(out, ATT_KEY);
+            SimpleTextUtil.write(out, entry.getKey(), scratch);
+            SimpleTextUtil.writeNewline(out);
+          
+            SimpleTextUtil.write(out, ATT_VALUE);
+            SimpleTextUtil.write(out, entry.getValue(), scratch);
+            SimpleTextUtil.writeNewline(out);
+          }
+        }
       }
     } finally {
       out.close();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Tue May 29 07:22:08 2012
@@ -514,7 +514,7 @@ class SimpleTextFieldsReader extends Fie
 
     public SimpleTextTerms(String field, long termsStart) throws IOException {
       this.termsStart = termsStart;
-      indexOptions = fieldInfos.fieldInfo(field).indexOptions;
+      indexOptions = fieldInfos.fieldInfo(field).getIndexOptions();
       loadTerms();
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java Tue May 29 07:22:08 2012
@@ -46,7 +46,7 @@ class SimpleTextFieldsWriter extends Fie
   final static BytesRef PAYLOAD      = new BytesRef("        payload ");
 
   public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
-    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentName, state.segmentSuffix);
+    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix);
     out = state.directory.createOutput(fileName, state.context);
   }
 
@@ -107,7 +107,7 @@ class SimpleTextFieldsWriter extends Fie
     private int lastEndOffset = -1;
 
     public SimpleTextPostingsWriter(FieldInfo field) {
-      this.indexOptions = field.indexOptions;
+      this.indexOptions = field.getIndexOptions();
       writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
       writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       //System.out.println("writeOffsets=" + writeOffsets);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java Tue May 29 07:22:08 2012
@@ -19,11 +19,11 @@ package org.apache.lucene.codecs.simplet
 
 import java.io.IOException;
 import java.util.BitSet;
-import java.util.Set;
+import java.util.Collection;
 
 import org.apache.lucene.codecs.LiveDocsFormat;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentInfoPerCommit;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -63,12 +63,12 @@ public class SimpleTextLiveDocsFormat ex
   }
 
   @Override
-  public Bits readLiveDocs(Directory dir, SegmentInfo info, IOContext context) throws IOException {
+  public Bits readLiveDocs(Directory dir, SegmentInfoPerCommit info, IOContext context) throws IOException {
     assert info.hasDeletions();
     BytesRef scratch = new BytesRef();
     CharsRef scratchUTF16 = new CharsRef();
     
-    String fileName = IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen());
+    String fileName = IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getDelGen());
     IndexInput in = null;
     boolean success = false;
     try {
@@ -105,12 +105,12 @@ public class SimpleTextLiveDocsFormat ex
   }
 
   @Override
-  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException {
+  public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfoPerCommit info, int newDelCount, IOContext context) throws IOException {
     BitSet set = ((SimpleTextBits) bits).bits;
     int size = bits.length();
     BytesRef scratch = new BytesRef();
     
-    String fileName = IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen());
+    String fileName = IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getNextDelGen());
     IndexOutput out = null;
     boolean success = false;
     try {
@@ -138,9 +138,9 @@ public class SimpleTextLiveDocsFormat ex
   }
 
   @Override
-  public void files(SegmentInfo info, Set<String> files) throws IOException {
+  public void files(SegmentInfoPerCommit info, Collection<String> files) throws IOException {
     if (info.hasDeletions()) {
-      files.add(IndexFileNames.fileNameFromGeneration(info.name, LIVEDOCS_EXTENSION, info.getDelGen()));
+      files.add(IndexFileNames.fileNameFromGeneration(info.info.name, LIVEDOCS_EXTENSION, info.getDelGen()));
     }
   }
   

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextNormsFormat.java Tue May 29 07:22:08 2012
@@ -19,8 +19,6 @@ package org.apache.lucene.codecs.simplet
 
 import java.io.IOException;
 import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Set;
 
 import org.apache.lucene.codecs.NormsFormat;
 import org.apache.lucene.codecs.PerDocConsumer;
@@ -30,12 +28,9 @@ import org.apache.lucene.index.DocValues
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.PerDocWriteState;
-import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
 
 /**
  * plain-text norms format.
@@ -49,18 +44,13 @@ public class SimpleTextNormsFormat exten
   
   @Override
   public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
-    return new SimpleTextNormsPerDocConsumer(state, NORMS_SEG_SUFFIX);
+    return new SimpleTextNormsPerDocConsumer(state);
   }
   
   @Override
   public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
     return new SimpleTextNormsPerDocProducer(state,
-        BytesRef.getUTF8SortedAsUnicodeComparator(), NORMS_SEG_SUFFIX);
-  }
-  
-  @Override
-  public void files(SegmentInfo info, Set<String> files) throws IOException {
-    SimpleTextNormsPerDocConsumer.files(info, files);
+        BytesRef.getUTF8SortedAsUnicodeComparator());
   }
   
   /**
@@ -74,8 +64,8 @@ public class SimpleTextNormsFormat exten
       SimpleTextPerDocProducer {
     
     public SimpleTextNormsPerDocProducer(SegmentReadState state,
-        Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
-      super(state, comp, segmentSuffix);
+        Comparator<BytesRef> comp) throws IOException {
+      super(state, comp, NORMS_SEG_SUFFIX);
     }
     
     @Override
@@ -105,9 +95,9 @@ public class SimpleTextNormsFormat exten
   public static class SimpleTextNormsPerDocConsumer extends
       SimpleTextPerDocConsumer {
     
-    public SimpleTextNormsPerDocConsumer(PerDocWriteState state,
-        String segmentSuffix) throws IOException {
-      super(state, segmentSuffix);
+    public SimpleTextNormsPerDocConsumer(PerDocWriteState state)
+      throws IOException {
+      super(state, NORMS_SEG_SUFFIX);
     }
     
     @Override
@@ -128,27 +118,8 @@ public class SimpleTextNormsFormat exten
     
     @Override
     public void abort() {
-      Set<String> files = new HashSet<String>();
-      filesInternal(state.fieldInfos, state.segmentName, files, segmentSuffix);
-      IOUtils.deleteFilesIgnoringExceptions(state.directory,
-          files.toArray(new String[0]));
-    }
-    
-    public static void files(SegmentInfo segmentInfo, Set<String> files)
-        throws IOException {
-      filesInternal(segmentInfo.getFieldInfos(), segmentInfo.name, files,
-          NORMS_SEG_SUFFIX);
-    }
-    
-    public static void filesInternal(FieldInfos fieldInfos, String segmentName,
-        Set<String> files, String segmentSuffix) {
-      for (FieldInfo fieldInfo : fieldInfos) {
-        if (fieldInfo.hasNorms()) {
-          String id = docValuesId(segmentName, fieldInfo.number);
-          files.add(IndexFileNames.segmentFileName(id, "",
-              segmentSuffix));
-        }
-      }
+      // We don't have to remove files here: IndexFileDeleter
+      // will do so
     }
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocConsumer.java Tue May 29 07:22:08 2012
@@ -1,4 +1,5 @@
 package org.apache.lucene.codecs.simpletext;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements. See the NOTICE file distributed with this
@@ -15,20 +16,14 @@ package org.apache.lucene.codecs.simplet
  * License for the specific language governing permissions and limitations under
  * the License.
  */
+
 import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.PerDocWriteState;
-import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.DocValues.Type;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.IOUtils;
 
 /**
  * @lucene.experimental
@@ -51,44 +46,17 @@ class SimpleTextPerDocConsumer extends P
   @Override
   public DocValuesConsumer addValuesField(Type type, FieldInfo field)
       throws IOException {
-    return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentName,
+    return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentInfo.name,
         field.number), state.directory, state.context, type, segmentSuffix);
   }
 
   @Override
   public void abort() {
-    Set<String> files = new HashSet<String>();
-    files(state.directory, state.fieldInfos, state.segmentName, files, segmentSuffix);
-    IOUtils.deleteFilesIgnoringExceptions(state.directory,
-        files.toArray(new String[0]));
-  }
-  
-  
-  static void files(SegmentInfo info, Set<String> files, String segmentSuffix) throws IOException {
-    files(info.dir, info.getFieldInfos(), info.name, files, segmentSuffix);
+    // We don't have to remove files here: IndexFileDeleter
+    // will do so
   }
   
   static String docValuesId(String segmentsName, int fieldId) {
     return segmentsName + "_" + fieldId;
   }
-
-  @SuppressWarnings("fallthrough")
-  private static void files(Directory dir, FieldInfos fieldInfos,
-      String segmentName, Set<String> files, String segmentSuffix) {
-    for (FieldInfo fieldInfo : fieldInfos) {
-      if (fieldInfo.hasDocValues()) {
-        String filename = docValuesId(segmentName, fieldInfo.number);
-        files.add(IndexFileNames.segmentFileName(filename, "",
-            segmentSuffix));
-        try {
-          assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
-              segmentSuffix));
-        } catch (IOException e) {
-          // don't throw checked exception - dir is only used in assert
-          throw new RuntimeException(e);
-        }
-      }
-    }
-  }
-
-}
\ No newline at end of file
+}

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPerDocProducer.java Tue May 29 07:22:08 2012
@@ -68,7 +68,7 @@ public class SimpleTextPerDocProducer ex
     this.segmentSuffix = segmentSuffix;
     if (anyDocValuesFields(state.fieldInfos)) {
       docValues = load(state.fieldInfos, state.segmentInfo.name,
-          state.segmentInfo.docCount, state.dir, state.context);
+                       state.segmentInfo.getDocCount(), state.dir, state.context);
     } else {
       docValues = new TreeMap<String, DocValues>();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java Tue May 29 07:22:08 2012
@@ -18,12 +18,10 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
-import java.util.Set;
 
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.IndexFileNames;
@@ -58,9 +56,4 @@ public class SimpleTextPostingsFormat ex
   static String getPostingsFileName(String segment, String segmentSuffix) {
     return IndexFileNames.segmentFileName(segment, segmentSuffix, POSTINGS_EXTENSION);
   }
-
-  @Override
-  public void files(SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
-    files.add(getPostingsFileName(segmentInfo.name, segmentSuffix));
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsFormat.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsFormat.java Tue May 29 07:22:08 2012
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
-import java.util.Set;
 
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
@@ -37,17 +36,12 @@ import org.apache.lucene.store.IOContext
 public class SimpleTextStoredFieldsFormat extends StoredFieldsFormat {
 
   @Override
-  public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
+  public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {;
     return new SimpleTextStoredFieldsReader(directory, si, fn, context);
   }
 
   @Override
-  public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
-    return new SimpleTextStoredFieldsWriter(directory, segment, context);
-  }
-
-  @Override
-  public void files(SegmentInfo info, Set<String> files) throws IOException {
-    SimpleTextStoredFieldsReader.files(info, files);
+  public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
+    return new SimpleTextStoredFieldsWriter(directory, si.name, context);
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java Tue May 29 07:22:08 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.simplet
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Set;
 
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.index.CorruptIndexException;
@@ -178,10 +177,6 @@ public class SimpleTextStoredFieldsReade
     }
   }
   
-  public static void files(SegmentInfo info, Set<String> files) throws IOException {
-    files.add(IndexFileNames.segmentFileName(info.name, "", SimpleTextStoredFieldsWriter.FIELDS_EXTENSION));
-  }
-  
   private void readLine() throws IOException {
     SimpleTextUtil.readLine(in, scratch);
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java Tue May 29 07:22:08 2012
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.store.Directory;
@@ -163,7 +164,7 @@ public class SimpleTextStoredFieldsWrite
   }
 
   @Override
-  public void finish(int numDocs) throws IOException {
+  public void finish(FieldInfos fis, int numDocs) throws IOException {
     if (numDocsWritten != numDocs) {
       throw new RuntimeException("mergeFields produced an invalid result: docCount is " + numDocs 
           + " but only saw " + numDocsWritten + " file=" + out.toString() + "; now aborting this merge to prevent index corruption");

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsFormat.java Tue May 29 07:22:08 2012
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
-import java.util.Set;
 
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.TermVectorsReader;
@@ -42,12 +41,7 @@ public class SimpleTextTermVectorsFormat
   }
 
   @Override
-  public TermVectorsWriter vectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
-    return new SimpleTextTermVectorsWriter(directory, segment, context);
-  }
-
-  @Override
-  public void files(SegmentInfo info, Set<String> files) throws IOException {
-    SimpleTextTermVectorsReader.files(info, files);
+  public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
+    return new SimpleTextTermVectorsWriter(directory, segmentInfo.name, context);
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Tue May 29 07:22:08 2012
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
-import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
@@ -200,13 +199,7 @@ public class SimpleTextTermVectorsReader
       offsets = null;
     }
   }
-  
-  public static void files(SegmentInfo info, Set<String> files) throws IOException {
-    if (info.getHasVectors()) {
-      files.add(IndexFileNames.segmentFileName(info.name, "", VECTORS_EXTENSION));
-    }
-  }
-  
+
   private void readLine() throws IOException {
     SimpleTextUtil.readLine(in, scratch);
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java Tue May 29 07:22:08 2012
@@ -22,6 +22,7 @@ import java.util.Comparator;
 
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
@@ -154,7 +155,7 @@ public class SimpleTextTermVectorsWriter
   }
 
   @Override
-  public void finish(int numDocs) throws IOException {
+  public void finish(FieldInfos fis, int numDocs) throws IOException {
     if (numDocsWritten != numDocs) {
       throw new RuntimeException("mergeVectors produced an invalid result: mergedDocs is " + numDocs + " but vec numDocs is " + numDocsWritten + " file=" + out.toString() + "; now aborting this merge to prevent index corruption");
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/document/DocumentStoredFieldVisitor.java Tue May 29 07:22:08 2012
@@ -63,11 +63,10 @@ public class DocumentStoredFieldVisitor 
   @Override
   public void stringField(FieldInfo fieldInfo, String value) throws IOException {
     final FieldType ft = new FieldType(TextField.TYPE_STORED);
-    ft.setStoreTermVectors(fieldInfo.storeTermVector);
-    ft.setStoreTermVectors(fieldInfo.storeTermVector);
-    ft.setIndexed(fieldInfo.isIndexed);
-    ft.setOmitNorms(fieldInfo.omitNorms);
-    ft.setIndexOptions(fieldInfo.indexOptions);
+    ft.setStoreTermVectors(fieldInfo.hasVectors());
+    ft.setIndexed(fieldInfo.isIndexed());
+    ft.setOmitNorms(fieldInfo.omitsNorms());
+    ft.setIndexOptions(fieldInfo.getIndexOptions());
     doc.add(new Field(fieldInfo.name, value, ft));
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Tue May 29 07:22:08 2012
@@ -194,9 +194,7 @@ public abstract class AtomicReader exten
 
   /**
    * Get the {@link FieldInfos} describing all fields in
-   * this reader.  NOTE: do not make any changes to the
-   * returned FieldInfos!
-   *
+   * this reader.
    * @lucene.experimental
    */
   public abstract FieldInfos getFieldInfos();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Tue May 29 07:22:08 2012
@@ -121,9 +121,9 @@ class BufferedDeletesStream {
     public final long gen;
 
     // If non-null, contains segments that are 100% deleted
-    public final List<SegmentInfo> allDeleted;
+    public final List<SegmentInfoPerCommit> allDeleted;
 
-    ApplyDeletesResult(boolean anyDeletes, long gen, List<SegmentInfo> allDeleted) {
+    ApplyDeletesResult(boolean anyDeletes, long gen, List<SegmentInfoPerCommit> allDeleted) {
       this.anyDeletes = anyDeletes;
       this.gen = gen;
       this.allDeleted = allDeleted;
@@ -131,9 +131,9 @@ class BufferedDeletesStream {
   }
 
   // Sorts SegmentInfos from smallest to biggest bufferedDelGen:
-  private static final Comparator<SegmentInfo> sortSegInfoByDelGen = new Comparator<SegmentInfo>() {
+  private static final Comparator<SegmentInfoPerCommit> sortSegInfoByDelGen = new Comparator<SegmentInfoPerCommit>() {
     @Override
-    public int compare(SegmentInfo si1, SegmentInfo si2) {
+    public int compare(SegmentInfoPerCommit si1, SegmentInfoPerCommit si2) {
       final long cmp = si1.getBufferedDeletesGen() - si2.getBufferedDeletesGen();
       if (cmp > 0) {
         return 1;
@@ -148,7 +148,7 @@ class BufferedDeletesStream {
   /** Resolves the buffered deleted Term/Query/docIDs, into
    *  actual deleted docIDs in the liveDocs MutableBits for
    *  each SegmentReader. */
-  public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List<SegmentInfo> infos) throws IOException {
+  public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List<SegmentInfoPerCommit> infos) throws IOException {
     final long t0 = System.currentTimeMillis();
 
     if (infos.size() == 0) {
@@ -168,7 +168,7 @@ class BufferedDeletesStream {
       infoStream.message("BD", "applyDeletes: infos=" + infos + " packetCount=" + deletes.size());
     }
 
-    List<SegmentInfo> infos2 = new ArrayList<SegmentInfo>();
+    List<SegmentInfoPerCommit> infos2 = new ArrayList<SegmentInfoPerCommit>();
     infos2.addAll(infos);
     Collections.sort(infos2, sortSegInfoByDelGen);
 
@@ -178,13 +178,13 @@ class BufferedDeletesStream {
     int infosIDX = infos2.size()-1;
     int delIDX = deletes.size()-1;
 
-    List<SegmentInfo> allDeleted = null;
+    List<SegmentInfoPerCommit> allDeleted = null;
 
     while (infosIDX >= 0) {
       //System.out.println("BD: cycle delIDX=" + delIDX + " infoIDX=" + infosIDX);
 
       final FrozenBufferedDeletes packet = delIDX >= 0 ? deletes.get(delIDX) : null;
-      final SegmentInfo info = infos2.get(infosIDX);
+      final SegmentInfoPerCommit info = infos2.get(infosIDX);
       final long segGen = info.getBufferedDeletesGen();
 
       if (packet != null && segGen < packet.delGen()) {
@@ -225,8 +225,8 @@ class BufferedDeletesStream {
           // already did that on flush:
           delCount += applyQueryDeletes(packet.queriesIterable(), rld, reader);
           final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
-          assert fullDelCount <= rld.info.docCount;
-          segAllDeletes = fullDelCount == rld.info.docCount;
+          assert fullDelCount <= rld.info.info.getDocCount();
+          segAllDeletes = fullDelCount == rld.info.info.getDocCount();
         } finally {
           rld.release(reader);
           readerPool.release(rld);
@@ -235,7 +235,7 @@ class BufferedDeletesStream {
 
         if (segAllDeletes) {
           if (allDeleted == null) {
-            allDeleted = new ArrayList<SegmentInfo>();
+            allDeleted = new ArrayList<SegmentInfoPerCommit>();
           }
           allDeleted.add(info);
         }
@@ -271,8 +271,8 @@ class BufferedDeletesStream {
             delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
             delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
             final int fullDelCount = rld.info.getDelCount() + rld.getPendingDeleteCount();
-            assert fullDelCount <= rld.info.docCount;
-            segAllDeletes = fullDelCount == rld.info.docCount;
+            assert fullDelCount <= rld.info.info.getDocCount();
+            segAllDeletes = fullDelCount == rld.info.info.getDocCount();
           } finally {   
             rld.release(reader);
             readerPool.release(rld);
@@ -281,7 +281,7 @@ class BufferedDeletesStream {
 
           if (segAllDeletes) {
             if (allDeleted == null) {
-              allDeleted = new ArrayList<SegmentInfo>();
+              allDeleted = new ArrayList<SegmentInfoPerCommit>();
             }
             allDeleted.add(info);
           }
@@ -316,7 +316,7 @@ class BufferedDeletesStream {
   public synchronized void prune(SegmentInfos segmentInfos) {
     assert checkDeleteStats();
     long minGen = Long.MAX_VALUE;
-    for(SegmentInfo info : segmentInfos) {
+    for(SegmentInfoPerCommit info : segmentInfos) {
       minGen = Math.min(info.getBufferedDeletesGen(), minGen);
     }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue May 29 07:22:08 2012
@@ -88,9 +88,6 @@ public class CheckIndex {
     /** Number of segments in the index. */
     public int numSegments;
 
-    /** String description of the version of the index. */
-    public String segmentFormat;
-
     /** Empty unless you passed specific segments list to check as optional 3rd argument.
      *  @see CheckIndex#checkIndex(List) */
     public List<String> segmentsChecked = new ArrayList<String>();
@@ -185,11 +182,6 @@ public class CheckIndex {
       /** Number of fields in this segment. */
       int numFields;
 
-      /** True if at least one of the fields in this segment
-       *  has position data
-       *  @see FieldType#setIndexOptions(org.apache.lucene.index.FieldInfo.IndexOptions) */
-      public boolean hasProx;
-
       /** Map that includes certain
        *  debugging details that IndexWriter records into
        *  each segment it creates */
@@ -369,8 +361,8 @@ public class CheckIndex {
     String oldSegs = null;
     boolean foundNonNullVersion = false;
     Comparator<String> versionComparator = StringHelper.getVersionComparator();
-    for (SegmentInfo si : sis) {
-      String version = si.getVersion();
+    for (SegmentInfoPerCommit si : sis) {
+      String version = si.info.getVersion();
       if (version == null) {
         // pre-3.1 segment
         oldSegs = "pre-3.1";
@@ -415,27 +407,8 @@ public class CheckIndex {
     String sFormat = "";
     boolean skip = false;
 
-    if (format == SegmentInfos.FORMAT_DIAGNOSTICS) {
-      sFormat = "FORMAT_DIAGNOSTICS [Lucene 2.9]";
-    } else if (format == SegmentInfos.FORMAT_HAS_VECTORS) {
-      sFormat = "FORMAT_HAS_VECTORS [Lucene 3.1]";
-    } else if (format == SegmentInfos.FORMAT_3_1) {
-      sFormat = "FORMAT_3_1 [Lucene 3.1+]";
-    } else if (format == SegmentInfos.FORMAT_4_0) {
-      sFormat = "FORMAT_4_0 [Lucene 4.0]";
-    } else if (format == SegmentInfos.FORMAT_CURRENT) {
-      throw new RuntimeException("BUG: You should update this tool!");
-    } else if (format < SegmentInfos.FORMAT_CURRENT) {
-      sFormat = "int=" + format + " [newer version of Lucene than this tool supports]";
-      skip = true;
-    } else if (format > SegmentInfos.FORMAT_MINIMUM) {
-      sFormat = "int=" + format + " [older version of Lucene than this tool supports]";
-      skip = true;
-    }
-
     result.segmentsFileName = segmentsFileName;
     result.numSegments = numSegments;
-    result.segmentFormat = sFormat;
     result.userData = sis.getUserData();
     String userDataString;
     if (sis.getUserData().size() > 0) {
@@ -482,50 +455,41 @@ public class CheckIndex {
     result.maxSegmentName = -1;
 
     for(int i=0;i<numSegments;i++) {
-      final SegmentInfo info = sis.info(i);
-      int segmentName = Integer.parseInt(info.name.substring(1), Character.MAX_RADIX);
+      final SegmentInfoPerCommit info = sis.info(i);
+      int segmentName = Integer.parseInt(info.info.name.substring(1), Character.MAX_RADIX);
       if (segmentName > result.maxSegmentName) {
         result.maxSegmentName = segmentName;
       }
-      if (onlySegments != null && !onlySegments.contains(info.name))
+      if (onlySegments != null && !onlySegments.contains(info.info.name)) {
         continue;
+      }
       Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
       result.segmentInfos.add(segInfoStat);
-      msg("  " + (1+i) + " of " + numSegments + ": name=" + info.name + " docCount=" + info.docCount);
-      segInfoStat.name = info.name;
-      segInfoStat.docCount = info.docCount;
+      msg("  " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
+      segInfoStat.name = info.info.name;
+      segInfoStat.docCount = info.info.getDocCount();
 
-      int toLoseDocCount = info.docCount;
+      int toLoseDocCount = info.info.getDocCount();
 
       SegmentReader reader = null;
 
       try {
-        final Codec codec = info.getCodec();
+        final Codec codec = info.info.getCodec();
         msg("    codec=" + codec);
         segInfoStat.codec = codec;
-        msg("    compound=" + info.getUseCompoundFile());
-        segInfoStat.compound = info.getUseCompoundFile();
-        msg("    hasProx=" + info.getHasProx());
-        segInfoStat.hasProx = info.getHasProx();
+        msg("    compound=" + info.info.getUseCompoundFile());
+        segInfoStat.compound = info.info.getUseCompoundFile();
         msg("    numFiles=" + info.files().size());
         segInfoStat.numFiles = info.files().size();
         segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
         msg("    size (MB)=" + nf.format(segInfoStat.sizeMB));
-        Map<String,String> diagnostics = info.getDiagnostics();
+        Map<String,String> diagnostics = info.info.getDiagnostics();
         segInfoStat.diagnostics = diagnostics;
         if (diagnostics.size() > 0) {
           msg("    diagnostics = " + diagnostics);
         }
 
-        final int docStoreOffset = info.getDocStoreOffset();
-        if (docStoreOffset != -1) {
-          msg("    docStoreOffset=" + docStoreOffset);
-          segInfoStat.docStoreOffset = docStoreOffset;
-          msg("    docStoreSegment=" + info.getDocStoreSegment());
-          segInfoStat.docStoreSegment = info.getDocStoreSegment();
-          msg("    docStoreIsCompoundFile=" + info.getDocStoreIsCompoundFile());
-          segInfoStat.docStoreCompoundFile = info.getDocStoreIsCompoundFile();
-        }
+        // TODO: we could append the info attributes() to the msg?
 
         if (info.hasDeletions()) {
           msg("    no deletions");
@@ -545,14 +509,14 @@ public class CheckIndex {
         final int numDocs = reader.numDocs();
         toLoseDocCount = numDocs;
         if (reader.hasDeletions()) {
-          if (reader.numDocs() != info.docCount - info.getDelCount()) {
-            throw new RuntimeException("delete count mismatch: info=" + (info.docCount - info.getDelCount()) + " vs reader=" + reader.numDocs());
+          if (reader.numDocs() != info.info.getDocCount() - info.getDelCount()) {
+            throw new RuntimeException("delete count mismatch: info=" + (info.info.getDocCount() - info.getDelCount()) + " vs reader=" + reader.numDocs());
           }
-          if ((info.docCount-reader.numDocs()) > reader.maxDoc()) {
-            throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.docCount-reader.numDocs()));
+          if ((info.info.getDocCount()-reader.numDocs()) > reader.maxDoc()) {
+            throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.info.getDocCount()-reader.numDocs()));
           }
-          if (info.docCount - numDocs != info.getDelCount()) {
-            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.docCount - numDocs));
+          if (info.info.getDocCount() - numDocs != info.getDelCount()) {
+            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
           }
           Bits liveDocs = reader.getLiveDocs();
           if (liveDocs == null) {
@@ -569,11 +533,11 @@ public class CheckIndex {
             }
           }
           
-          segInfoStat.numDeleted = info.docCount - numDocs;
+          segInfoStat.numDeleted = info.info.getDocCount() - numDocs;
           msg("OK [" + (segInfoStat.numDeleted) + " deleted docs]");
         } else {
           if (info.getDelCount() != 0) {
-            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.docCount - numDocs));
+            throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
           }
           Bits liveDocs = reader.getLiveDocs();
           if (liveDocs != null) {
@@ -586,8 +550,9 @@ public class CheckIndex {
           }
           msg("OK");
         }
-        if (reader.maxDoc() != info.docCount)
-          throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.docCount);
+        if (reader.maxDoc() != info.info.getDocCount()) {
+          throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
+        }
 
         // Test getFieldInfos()
         if (infoStream != null) {
@@ -609,7 +574,7 @@ public class CheckIndex {
         // Test Term Vectors
         segInfoStat.termVectorStatus = testTermVectors(fieldInfos, info, reader, nf);
         
-        segInfoStat.docValuesStatus = testDocValues(info, reader);
+        segInfoStat.docValuesStatus = testDocValues(info, fieldInfos, reader);
 
         // Rethrow the first exception we encountered
         //  This will cause stats for failed segments to be incremented properly
@@ -706,8 +671,7 @@ public class CheckIndex {
    * checks Fields api is consistent with itself.
    * searcher is optional, to verify with queries. Can be null.
    */
-  // TODO: cutover term vectors to this!
-  private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher) throws IOException {
+  private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, IndexSearcher searcher, boolean doPrint) throws IOException {
     // TODO: we should probably return our own stats thing...?!
     
     final Status.TermIndexStatus status = new Status.TermIndexStatus();
@@ -741,7 +705,7 @@ public class CheckIndex {
       if (fi == null) {
         throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, no fieldInfos for: " + field);
       }
-      if (!fi.isIndexed) {
+      if (!fi.isIndexed()) {
         throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, isIndexed == false for: " + field);
       }
       
@@ -1131,8 +1095,10 @@ public class CheckIndex {
     if (status.termCount != uniqueTermCountAllFields) {
       throw new RuntimeException("termCount mismatch " + uniqueTermCountAllFields + " vs " + (status.termCount));
     }
-    
-    msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
+
+    if (doPrint) {
+      msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
+    }
     
     if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
       for(Map.Entry<String,BlockTreeTermsReader.Stats> ent : status.blockTreeStats.entrySet()) {
@@ -1163,13 +1129,13 @@ public class CheckIndex {
       }
 
       final Fields fields = reader.fields();
-      status = checkFields(fields, liveDocs, maxDoc, fieldInfos, is);
+      status = checkFields(fields, liveDocs, maxDoc, fieldInfos, is, true);
       if (liveDocs != null) {
         if (infoStream != null) {
           infoStream.print("    test (ignoring deletes): terms, freq, prox...");
         }
         // TODO: can we make a IS that ignores all deletes?
-        checkFields(fields, null, maxDoc, fieldInfos, null);
+        checkFields(fields, null, maxDoc, fieldInfos, null, true);
       }
     } catch (Throwable e) {
       msg("ERROR: " + e);
@@ -1186,7 +1152,7 @@ public class CheckIndex {
   /**
    * Test stored fields for a segment.
    */
-  private Status.StoredFieldStatus testStoredFields(SegmentInfo info, SegmentReader reader, NumberFormat format) {
+  private Status.StoredFieldStatus testStoredFields(SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
     final Status.StoredFieldStatus status = new Status.StoredFieldStatus();
 
     try {
@@ -1196,7 +1162,7 @@ public class CheckIndex {
 
       // Scan stored fields for all documents
       final Bits liveDocs = reader.getLiveDocs();
-      for (int j = 0; j < info.docCount; ++j) {
+      for (int j = 0; j < info.info.getDocCount(); ++j) {
         // Intentionally pull even deleted documents to
         // make sure they too are not corrupt:
         Document doc = reader.document(j);
@@ -1310,14 +1276,14 @@ public class CheckIndex {
     }
   }
   
-  private Status.DocValuesStatus testDocValues(SegmentInfo info,
-      SegmentReader reader) {
+  private Status.DocValuesStatus testDocValues(SegmentInfoPerCommit info,
+                                               FieldInfos fieldInfos,
+                                               SegmentReader reader) {
     final Status.DocValuesStatus status = new Status.DocValuesStatus();
     try {
       if (infoStream != null) {
         infoStream.print("    test: DocValues........");
       }
-      final FieldInfos fieldInfos = info.getFieldInfos();
       for (FieldInfo fieldInfo : fieldInfos) {
         if (fieldInfo.hasDocValues()) {
           status.totalValueFields++;
@@ -1345,7 +1311,7 @@ public class CheckIndex {
   /**
    * Test term vectors for a segment.
    */
-  private Status.TermVectorStatus testTermVectors(FieldInfos fieldInfos, SegmentInfo info, SegmentReader reader, NumberFormat format) {
+  private Status.TermVectorStatus testTermVectors(FieldInfos fieldInfos, SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
     final Status.TermVectorStatus status = new Status.TermVectorStatus();
 
     final Bits onlyDocIsDeleted = new FixedBitSet(1);
@@ -1375,7 +1341,7 @@ public class CheckIndex {
       TermsEnum termsEnum = null;
       TermsEnum postingsTermsEnum = null;
 
-      for (int j = 0; j < info.docCount; ++j) {
+      for (int j = 0; j < info.info.getDocCount(); ++j) {
         // Intentionally pull/visit (but don't count in
         // stats) deleted documents to make sure they too
         // are not corrupt:
@@ -1386,10 +1352,10 @@ public class CheckIndex {
 
         if (tfv != null) {
           // First run with no deletions:
-          checkFields(tfv, null, 1, fieldInfos, null);
+          checkFields(tfv, null, 1, fieldInfos, null, false);
 
           // Again, with the one doc deleted:
-          checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, null);
+          checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, null, false);
 
           // Only agg stats if the doc is live:
           final boolean doStats = liveDocs == null || liveDocs.get(j);
@@ -1406,7 +1372,7 @@ public class CheckIndex {
 
             // Make sure FieldInfo thinks this field is vector'd:
             final FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
-            if (!fieldInfo.storeTermVector) {
+            if (!fieldInfo.hasVectors()) {
               throw new RuntimeException("docID=" + j + " has term vectors for field=" + field + " but FieldInfo has storeTermVector=false");
             }
 
@@ -1594,7 +1560,7 @@ public class CheckIndex {
     if (result.partial)
       throw new IllegalArgumentException("can only fix an index that was fully checked (this status checked a subset of segments)");
     result.newSegments.changed();
-    result.newSegments.commit(result.dir, codec);
+    result.newSegments.commit(result.dir);
   }
 
   private static boolean assertsOn;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java Tue May 29 07:22:08 2012
@@ -20,7 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 abstract class DocConsumer {
-  abstract void processDocument(FieldInfos fieldInfos) throws IOException;
+  abstract void processDocument(FieldInfos.Builder fieldInfos) throws IOException;
   abstract void finishDocument() throws IOException;
   abstract void flush(final SegmentWriteState state) throws IOException;
   abstract void abort();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldConsumer.java Tue May 29 07:22:08 2012
@@ -23,7 +23,7 @@ import java.util.Map;
 abstract class DocFieldConsumer {
   /** Called when DocumentsWriterPerThread decides to create a new
    *  segment */
-  abstract void flush(Map<FieldInfo, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
+  abstract void flush(Map<String, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException;
 
   /** Called when an aborting exception is hit */
   abstract void abort();

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Tue May 29 07:22:08 2012
@@ -71,28 +71,28 @@ final class DocFieldProcessor extends Do
   @Override
   public void flush(SegmentWriteState state) throws IOException {
 
-    Map<FieldInfo, DocFieldConsumerPerField> childFields = new HashMap<FieldInfo, DocFieldConsumerPerField>();
+    Map<String,DocFieldConsumerPerField> childFields = new HashMap<String,DocFieldConsumerPerField>();
     Collection<DocFieldConsumerPerField> fields = fields();
     for (DocFieldConsumerPerField f : fields) {
-      childFields.put(f.getFieldInfo(), f);
+      childFields.put(f.getFieldInfo().name, f);
     }
 
     fieldsWriter.flush(state);
     consumer.flush(childFields, state);
 
     for (DocValuesConsumerAndDocID consumer : docValues.values()) {
-      consumer.docValuesConsumer.finish(state.numDocs);
+      consumer.docValuesConsumer.finish(state.segmentInfo.getDocCount());
     }
+    
+    // close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
+    IOUtils.close(perDocConsumer);
 
     // Important to save after asking consumer to flush so
     // consumer can alter the FieldInfo* if necessary.  EG,
     // FreqProxTermsWriter does this with
     // FieldInfo.storePayload.
     FieldInfosWriter infosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
-    infosWriter.write(state.directory, state.segmentName, state.fieldInfos, IOContext.DEFAULT);
-
-    // close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
-    IOUtils.close(perDocConsumer);
+    infosWriter.write(state.directory, state.segmentInfo.name, state.fieldInfos, IOContext.DEFAULT);
   }
 
   @Override
@@ -203,7 +203,7 @@ final class DocFieldProcessor extends Do
   }
 
   @Override
-  public void processDocument(FieldInfos fieldInfos) throws IOException {
+  public void processDocument(FieldInfos.Builder fieldInfos) throws IOException {
 
     consumer.startDocument();
     fieldsWriter.startDocument();
@@ -342,7 +342,8 @@ final class DocFieldProcessor extends Do
       }
     }
     DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
-    fieldInfo.setDocValuesType(valueType, false);
+    assert fieldInfo.getDocValuesType() == null || fieldInfo.getDocValuesType() == valueType;
+    fieldInfo.setDocValuesType(valueType);
 
     docValuesConsumerAndDocID = new DocValuesConsumerAndDocID(docValuesConsumer);
     docValuesConsumerAndDocID.docID = docState.docID;

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocInverter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocInverter.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocInverter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocInverter.java Tue May 29 07:22:08 2012
@@ -39,12 +39,12 @@ final class DocInverter extends DocField
   }
 
   @Override
-  void flush(Map<FieldInfo, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
+  void flush(Map<String, DocFieldConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
 
-    Map<FieldInfo, InvertedDocConsumerPerField> childFieldsToFlush = new HashMap<FieldInfo, InvertedDocConsumerPerField>();
-    Map<FieldInfo, InvertedDocEndConsumerPerField> endChildFieldsToFlush = new HashMap<FieldInfo, InvertedDocEndConsumerPerField>();
+    Map<String, InvertedDocConsumerPerField> childFieldsToFlush = new HashMap<String, InvertedDocConsumerPerField>();
+    Map<String, InvertedDocEndConsumerPerField> endChildFieldsToFlush = new HashMap<String, InvertedDocEndConsumerPerField>();
 
-    for (Map.Entry<FieldInfo, DocFieldConsumerPerField> fieldToFlush : fieldsToFlush.entrySet()) {
+    for (Map.Entry<String, DocFieldConsumerPerField> fieldToFlush : fieldsToFlush.entrySet()) {
       DocInverterPerField perField = (DocInverterPerField) fieldToFlush.getValue();
       childFieldsToFlush.put(fieldToFlush.getKey(), perField.consumer);
       endChildFieldsToFlush.put(fieldToFlush.getKey(), perField.endConsumer);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java Tue May 29 07:22:08 2012
@@ -28,7 +28,7 @@ import org.apache.lucene.index.Documents
 import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
 import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain;
 import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
-import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
+import org.apache.lucene.index.FieldInfos.FieldNumbers;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.AlreadyClosedException;
@@ -133,7 +133,7 @@ final class DocumentsWriter {
   final DocumentsWriterFlushControl flushControl;
   
   final Codec codec;
-  DocumentsWriter(Codec codec, IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers,
+  DocumentsWriter(Codec codec, IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumbers globalFieldNumbers,
       BufferedDeletesStream bufferedDeletesStream) throws IOException {
     this.codec = codec;
     this.directory = directory;
@@ -494,7 +494,8 @@ final class DocumentsWriter {
   private void publishFlushedSegment(FlushedSegment newSegment, FrozenBufferedDeletes globalPacket)
       throws IOException {
     assert newSegment != null;
-    final SegmentInfo segInfo = indexWriter.prepareFlushedSegment(newSegment);
+    assert newSegment.segmentInfo != null;
+    final SegmentInfoPerCommit segInfo = indexWriter.prepareFlushedSegment(newSegment);
     final BufferedDeletes deletes = newSegment.segmentDeletes;
     if (infoStream.isEnabled("DW")) {
       infoStream.message("DW", Thread.currentThread().getName() + ": publishFlushedSegment seg-private deletes=" + deletes);  

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java Tue May 29 07:22:08 2012
@@ -170,7 +170,7 @@ class DocumentsWriterFlushQueue {
     protected abstract boolean canPublish();
   }
   
-  static final class GlobalDeletesTicket extends FlushTicket{
+  static final class GlobalDeletesTicket extends FlushTicket {
 
     protected GlobalDeletesTicket(FrozenBufferedDeletes frozenDeletes) {
       super(frozenDeletes);

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Tue May 29 07:22:08 2012
@@ -17,11 +17,9 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
-import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
-
 import java.io.IOException;
 import java.text.NumberFormat;
+import java.util.HashSet;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.codecs.Codec;
@@ -30,12 +28,17 @@ import org.apache.lucene.search.similari
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.Counter;
+import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
 import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.MutableBits;
+import org.apache.lucene.util.RamUsageEstimator;
+
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
 class DocumentsWriterPerThread {
 
@@ -112,14 +115,16 @@ class DocumentsWriterPerThread {
   }
 
   static class FlushedSegment {
-    final SegmentInfo segmentInfo;
+    final SegmentInfoPerCommit segmentInfo;
+    final FieldInfos fieldInfos;
     final BufferedDeletes segmentDeletes;
     final MutableBits liveDocs;
     final int delCount;
 
-    private FlushedSegment(SegmentInfo segmentInfo,
+    private FlushedSegment(SegmentInfoPerCommit segmentInfo, FieldInfos fieldInfos,
                            BufferedDeletes segmentDeletes, MutableBits liveDocs, int delCount) {
       this.segmentInfo = segmentInfo;
+      this.fieldInfos = fieldInfos;
       this.segmentDeletes = segmentDeletes;
       this.liveDocs = liveDocs;
       this.delCount = delCount;
@@ -157,7 +162,8 @@ class DocumentsWriterPerThread {
   final DocumentsWriter parent;
   final Codec codec;
   final IndexWriter writer;
-  final Directory directory;
+  final TrackingDirectoryWrapper directory;
+  final Directory directoryOrig;
   final DocState docState;
   final DocConsumer consumer;
   final Counter bytesUsed;
@@ -165,11 +171,11 @@ class DocumentsWriterPerThread {
   SegmentWriteState flushState;
   //Deletes for our still-in-RAM (to be flushed next) segment
   BufferedDeletes pendingDeletes;  
-  String segment;     // Current segment we are working on
+  SegmentInfo segmentInfo;     // Current segment we are working on
   boolean aborting = false;   // True if an abort is pending
   boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting
 
-  private FieldInfos fieldInfos;
+  private FieldInfos.Builder fieldInfos;
   private final InfoStream infoStream;
   private int numDocsInRAM;
   private int flushedDocCount;
@@ -180,8 +186,9 @@ class DocumentsWriterPerThread {
 
   
   public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent,
-      FieldInfos fieldInfos, IndexingChain indexingChain) {
-    this.directory = directory;
+      FieldInfos.Builder fieldInfos, IndexingChain indexingChain) {
+    this.directoryOrig = directory;
+    this.directory = new TrackingDirectoryWrapper(directory);
     this.parent = parent;
     this.fieldInfos = fieldInfos;
     this.writer = parent.indexWriter;
@@ -196,8 +203,8 @@ class DocumentsWriterPerThread {
     initialize();
   }
   
-  public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos fieldInfos) {
-    this(other.directory, other.parent, fieldInfos, other.parent.chain);
+  public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos.Builder fieldInfos) {
+    this(other.directoryOrig, other.parent, fieldInfos, other.parent.chain);
   }
   
   void initialize() {
@@ -223,17 +230,11 @@ class DocumentsWriterPerThread {
     docState.doc = doc;
     docState.analyzer = analyzer;
     docState.docID = numDocsInRAM;
-    if (segment == null) {
-      // this call is synchronized on IndexWriter.segmentInfos
-      segment = writer.newSegmentName();
-      assert numDocsInRAM == 0;
-      if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
-        infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);  
-      }
-      
+    if (segmentInfo == null) {
+      initSegmentInfo();
     }
     if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
-      infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
+      infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segmentInfo.name);
     }
     boolean success = false;
     try {
@@ -265,21 +266,26 @@ class DocumentsWriterPerThread {
     }
     finishDocument(delTerm);
   }
+
+  private void initSegmentInfo() {
+    String segment = writer.newSegmentName();
+    segmentInfo = new SegmentInfo(directoryOrig, Constants.LUCENE_MAIN_VERSION, segment, -1,
+                                  false, codec, null, null);
+    assert numDocsInRAM == 0;
+    if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
+      infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);  
+    }
+  }
   
   public int updateDocuments(Iterable<? extends Iterable<? extends IndexableField>> docs, Analyzer analyzer, Term delTerm) throws IOException {
     assert writer.testPoint("DocumentsWriterPerThread addDocuments start");
     assert deleteQueue != null;
     docState.analyzer = analyzer;
-    if (segment == null) {
-      // this call is synchronized on IndexWriter.segmentInfos
-      segment = writer.newSegmentName();
-      assert numDocsInRAM == 0;
-      if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
-        infoStream.message("DWPT", Thread.currentThread().getName() + " init seg=" + segment + " delQueue=" + deleteQueue);  
-      }
+    if (segmentInfo == null) {
+      initSegmentInfo();
     }
     if (INFO_VERBOSE && infoStream.isEnabled("DWPT")) {
-      infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segment);
+      infoStream.message("DWPT", Thread.currentThread().getName() + " update delTerm=" + delTerm + " docID=" + docState.docID + " seg=" + segmentInfo.name);
     }
     int docCount = 0;
     try {
@@ -405,15 +411,12 @@ class DocumentsWriterPerThread {
     return numDocsInRAM;
   }
 
-  Codec getCodec() {
-    return flushState.codec;
-  }
-
   /** Reset after a flush */
   private void doAfterFlush() throws IOException {
-    segment = null;
+    segmentInfo = null;
     consumer.doAfterFlush();
-    fieldInfos = FieldInfos.from(fieldInfos);
+    directory.getCreatedFiles().clear();
+    fieldInfos = new FieldInfos.Builder(fieldInfos.globalFieldNumbers);
     parent.subtractFlushedNumDocs(numDocsInRAM);
     numDocsInRAM = 0;
   }
@@ -441,10 +444,12 @@ class DocumentsWriterPerThread {
   FlushedSegment flush() throws IOException {
     assert numDocsInRAM > 0;
     assert deleteSlice == null : "all deletes must be applied in prepareFlush";
-    flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos,
-        numDocsInRAM, writer.getConfig().getTermIndexInterval(),
-        codec, pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
+    segmentInfo.setDocCount(numDocsInRAM);
+    flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(),
+        writer.getConfig().getTermIndexInterval(),
+        pendingDeletes, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
     final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.;
+
     // Apply delete-by-docID now (delete-byDocID only
     // happens when an exception is hit processing that
     // doc, eg if analyzer has some problem w/ the text):
@@ -459,7 +464,7 @@ class DocumentsWriterPerThread {
     }
 
     if (infoStream.isEnabled("DWPT")) {
-      infoStream.message("DWPT", "flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM);
+      infoStream.message("DWPT", "flush postings as segment " + flushState.segmentInfo.name + " numDocs=" + numDocsInRAM);
     }
 
     if (aborting) {
@@ -474,14 +479,22 @@ class DocumentsWriterPerThread {
     try {
       consumer.flush(flushState);
       pendingDeletes.terms.clear();
-      final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
+      segmentInfo.setFiles(new HashSet<String>(directory.getCreatedFiles()));
+
+      final SegmentInfoPerCommit segmentInfoPerCommit = new SegmentInfoPerCommit(segmentInfo, 0, -1L);
       if (infoStream.isEnabled("DWPT")) {
-        infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs");
-        infoStream.message("DWPT", "new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
-        infoStream.message("DWPT", "flushedFiles=" + newSegment.files());
-        infoStream.message("DWPT", "flushed codec=" + newSegment.getCodec());
+        infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.segmentInfo.getDocCount() - flushState.delCountOnFlush)) + " deleted docs");
+        infoStream.message("DWPT", "new segment has " +
+                           (flushState.fieldInfos.hasVectors() ? "vectors" : "no vectors") + "; " +
+                           (flushState.fieldInfos.hasNorms() ? "norms" : "no norms") + "; " + 
+                           (flushState.fieldInfos.hasDocValues() ? "docValues" : "no docValues") + "; " + 
+                           (flushState.fieldInfos.hasProx() ? "prox" : "no prox") + "; " + 
+                           (flushState.fieldInfos.hasFreq() ? "freqs" : "no freqs"));
+        infoStream.message("DWPT", "flushedFiles=" + segmentInfoPerCommit.files());
+        infoStream.message("DWPT", "flushed codec=" + codec);
       }
-      flushedDocCount += flushState.numDocs;
+
+      flushedDocCount += flushState.segmentInfo.getDocCount();
 
       final BufferedDeletes segmentDeletes;
       if (pendingDeletes.queries.isEmpty()) {
@@ -493,21 +506,26 @@ class DocumentsWriterPerThread {
       }
 
       if (infoStream.isEnabled("DWPT")) {
-        final double newSegmentSize = newSegment.sizeInBytes()/1024./1024.;
-        infoStream.message("DWPT", "flushed: segment=" + newSegment + 
+        final double newSegmentSize = segmentInfo.sizeInBytes()/1024./1024.;
+        infoStream.message("DWPT", "flushed: segment=" + segmentInfo.name + 
                 " ramUsed=" + nf.format(startMBUsed) + " MB" +
                 " newFlushedSize(includes docstores)=" + nf.format(newSegmentSize) + " MB" +
                 " docs/MB=" + nf.format(flushedDocCount / newSegmentSize));
       }
+
+      assert segmentInfo != null;
+
+      FlushedSegment fs = new FlushedSegment(segmentInfoPerCommit, flushState.fieldInfos,
+                                             segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
       doAfterFlush();
       success = true;
 
-      return new FlushedSegment(newSegment, segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
+      return fs;
     } finally {
       if (!success) {
-        if (segment != null) {
+        if (segmentInfo != null) {
           synchronized(parent.indexWriter) {
-            parent.indexWriter.deleter.refresh(segment);
+            parent.indexWriter.deleter.refresh(segmentInfo.name);
           }
         }
         abort();
@@ -515,9 +533,9 @@ class DocumentsWriterPerThread {
     }
   }
 
-  /** Get current segment name we are writing. */
-  String getSegment() {
-    return segment;
+  /** Get current segment info we are writing. */
+  SegmentInfo getSegmentInfo() {
+    return segmentInfo;
   }
 
   long bytesUsed() {
@@ -550,14 +568,14 @@ class DocumentsWriterPerThread {
   }
 
   PerDocWriteState newPerDocWriteState(String segmentSuffix) {
-    assert segment != null;
-    return new PerDocWriteState(infoStream, directory, segment, fieldInfos, bytesUsed, segmentSuffix, IOContext.DEFAULT);
+    assert segmentInfo != null;
+    return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT);
   }
   
   @Override
   public String toString() {
     return "DocumentsWriterPerThread [pendingDeletes=" + pendingDeletes
-        + ", segment=" + segment + ", aborting=" + aborting + ", numDocsInRAM="
+      + ", segment=" + (segmentInfo != null ? segmentInfo.name : "null") + ", aborting=" + aborting + ", numDocsInRAM="
         + numDocsInRAM + ", deleteQueue=" + deleteQueue + "]";
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java?rev=1343568&r1=1343567&r2=1343568&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java Tue May 29 07:22:08 2012
@@ -18,7 +18,7 @@ package org.apache.lucene.index;
 
 import java.util.concurrent.locks.ReentrantLock;
 
-import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
+import org.apache.lucene.index.FieldInfos.FieldNumbers;
 import org.apache.lucene.util.SetOnce;
 
 /**
@@ -121,7 +121,7 @@ abstract class DocumentsWriterPerThreadP
 
   private final ThreadState[] threadStates;
   private volatile int numThreadStatesActive;
-  private final SetOnce<FieldNumberBiMap> globalFieldMap = new SetOnce<FieldNumberBiMap>();
+  private final SetOnce<FieldNumbers> globalFieldMap = new SetOnce<FieldNumbers>();
   private final SetOnce<DocumentsWriter> documentsWriter = new SetOnce<DocumentsWriter>();
   
   /**
@@ -135,11 +135,11 @@ abstract class DocumentsWriterPerThreadP
     numThreadStatesActive = 0;
   }
 
-  void initialize(DocumentsWriter documentsWriter, FieldNumberBiMap globalFieldMap, IndexWriterConfig config) {
+  void initialize(DocumentsWriter documentsWriter, FieldNumbers globalFieldMap, IndexWriterConfig config) {
     this.documentsWriter.set(documentsWriter); // thread pool is bound to DW
     this.globalFieldMap.set(globalFieldMap);
     for (int i = 0; i < threadStates.length; i++) {
-      final FieldInfos infos = new FieldInfos(globalFieldMap);
+      final FieldInfos.Builder infos = new FieldInfos.Builder(globalFieldMap);
       threadStates[i] = new ThreadState(new DocumentsWriterPerThread(documentsWriter.directory, documentsWriter, infos, documentsWriter.chain));
     }
   }
@@ -228,7 +228,7 @@ abstract class DocumentsWriterPerThreadP
     assert globalFieldMap.get() != null;
     final DocumentsWriterPerThread dwpt = threadState.dwpt;
     if (!closed) {
-      final FieldInfos infos = new FieldInfos(globalFieldMap.get());
+      final FieldInfos.Builder infos = new FieldInfos.Builder(globalFieldMap.get());
       final DocumentsWriterPerThread newDwpt = new DocumentsWriterPerThread(dwpt, infos);
       newDwpt.initialize();
       threadState.resetWriter(newDwpt);



Mime
View raw message