You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2011/11/03 00:55:04 UTC

svn commit: r1196874 [2/2] - in /lucene/dev/branches/lucene2621/lucene: contrib/misc/src/java/org/apache/lucene/index/codecs/appending/ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/index/codecs/ src/java/org/apache/lucene/index/codecs/l...

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/index/values/Bytes.java Wed Nov  2 23:55:02 2011
@@ -494,8 +494,7 @@ public final class Bytes {
     
     protected void releaseResources() {
       hash.close();
-      bytesUsed
-      .addAndGet((-docToEntry.length) * RamUsageEstimator.NUM_BYTES_INT);
+      bytesUsed.addAndGet((-docToEntry.length) * RamUsageEstimator.NUM_BYTES_INT);
       docToEntry = null;
     }
     

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileDirectory.java Wed Nov  2 23:55:02 2011
@@ -106,7 +106,9 @@ public final class CompoundFileDirectory
               numEntries);
           for (int i = 0; i < numEntries; i++) {
             final FileEntry fileEntry = new FileEntry();
-            mapping.put(input.readString(), fileEntry);
+            final String id = input.readString();
+            assert !mapping.containsKey(id): "id=" + id + " was written multiple times in the CFS";
+            mapping.put(id, fileEntry);
             fileEntry.offset = input.readLong();
             fileEntry.length = input.readLong();
           }
@@ -170,6 +172,9 @@ public final class CompoundFileDirectory
       
       entry = new FileEntry();
       entry.offset = offset;
+
+      assert !entries.containsKey(id);
+
       entries.put(id, entry);
     }
     
@@ -271,7 +276,7 @@ public final class CompoundFileDirectory
   public long fileLength(String name) throws IOException {
     ensureOpen();
     if (this.writer != null) {
-      return writer.fileLenght(name);
+      return writer.fileLength(name);
     }
     FileEntry e = entries.get(IndexFileNames.stripSegmentName(name));
     if (e == null)
@@ -323,4 +328,9 @@ public final class CompoundFileDirectory
       }
     };
   }
+
+  @Override
+  public String toString() {
+    return "CompoundFileDirectory(file=\"" + fileName + "\" in dir=" + directory + ")";
+  }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/java/org/apache/lucene/store/CompoundFileWriter.java Wed Nov  2 23:55:02 2011
@@ -22,9 +22,11 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Queue;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.index.IndexFileNames;
@@ -219,8 +221,12 @@ final class CompoundFileWriter implement
       IndexOutput entryOut) throws IOException {
     entryOut.writeInt(ENTRY_FORMAT_CURRENT);
     entryOut.writeVInt(entries.size());
+    final Set<String> seenIDs = new HashSet<String>();
     for (FileEntry fe : entries) {
-      entryOut.writeString(IndexFileNames.stripSegmentName(fe.file));
+      final String id = IndexFileNames.stripSegmentName(fe.file);
+      assert !seenIDs.contains(id): "file=\"" + fe.file + "\" maps to id=\"" + id + "\", which was written more than once";
+      seenIDs.add(id);
+      entryOut.writeString(id);
       entryOut.writeLong(fe.offset);
       entryOut.writeLong(fe.length);
     }
@@ -284,7 +290,7 @@ final class CompoundFileWriter implement
     }
   }
 
-  long fileLenght(String name) throws IOException {
+  long fileLength(String name) throws IOException {
     FileEntry fileEntry = entries.get(name);
     if (fileEntry == null) {
       throw new FileNotFoundException(name + " does not exist");

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/RandomCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/RandomCodec.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/RandomCodec.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/RandomCodec.java Wed Nov  2 23:55:02 2011
@@ -65,6 +65,8 @@ public class RandomCodec extends Lucene4
         codec = formats.get(Math.abs(perFieldSeed ^ name.toUpperCase(Locale.ENGLISH).hashCode()) % formats.size());
       }
       previousMappings.put(name, codec);
+      // Safety:
+      assert previousMappings.size() < 10000;
     }
     return codec;
   }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/lucene40ords/Lucene40WithOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/lucene40ords/Lucene40WithOrds.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/lucene40ords/Lucene40WithOrds.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/lucene40ords/Lucene40WithOrds.java Wed Nov  2 23:55:02 2011
@@ -89,7 +89,7 @@ public class Lucene40WithOrds extends Po
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
+    PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
     TermsIndexReaderBase indexReader;
 
     boolean success = false;
@@ -99,7 +99,7 @@ public class Lucene40WithOrds extends Po
                                                  state.segmentInfo.name,
                                                  state.termsIndexDivisor,
                                                  BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                 state.formatId, state.context);
+                                                 state.segmentSuffix, state.context);
       success = true;
     } finally {
       if (!success) {
@@ -116,7 +116,7 @@ public class Lucene40WithOrds extends Po
                                                 postings,
                                                 state.context,
                                                 TERMS_CACHE_SIZE,
-                                                state.formatId);
+                                                state.segmentSuffix);
       success = true;
       return ret;
     } finally {
@@ -137,9 +137,9 @@ public class Lucene40WithOrds extends Po
   static final String PROX_EXTENSION = "prx";
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
-    Lucene40PostingsReader.files(dir, segmentInfo, id, files);
-    BlockTermsReader.files(dir, segmentInfo, id, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, id, files);
+  public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
+    Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
+    BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Wed Nov  2 23:55:02 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs.m
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
@@ -30,8 +29,6 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
-import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
-import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.intblock.FixedIntBlockIndexInput;
@@ -39,8 +36,6 @@ import org.apache.lucene.index.codecs.in
 import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.BlockTermsReader;
@@ -165,7 +160,7 @@ public class MockFixedIntBlockPostingsFo
     PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
                                                               state.segmentInfo,
                                                               state.context,
-                                                              new MockIntFactory(blockSize), state.formatId);
+                                                              new MockIntFactory(blockSize), state.segmentSuffix);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;
@@ -174,7 +169,7 @@ public class MockFixedIntBlockPostingsFo
                                                        state.fieldInfos,
                                                        state.segmentInfo.name,
                                                        state.termsIndexDivisor,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(), state.formatId,
+                                                       BytesRef.getUTF8SortedAsUnicodeComparator(), state.segmentSuffix,
                                                        IOContext.DEFAULT);
       success = true;
     } finally {
@@ -192,7 +187,7 @@ public class MockFixedIntBlockPostingsFo
                                                 postingsReader,
                                                 state.context,
                                                 Lucene40PostingsFormat.TERMS_CACHE_SIZE,
-                                                state.formatId);
+                                                state.segmentSuffix);
       success = true;
       return ret;
     } finally {
@@ -207,9 +202,9 @@ public class MockFixedIntBlockPostingsFo
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int formatId, Set<String> files) throws IOException {
-    SepPostingsReader.files(segmentInfo, formatId, files);
-    BlockTermsReader.files(dir, segmentInfo, formatId, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, formatId, files);
+  public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
+    SepPostingsReader.files(segmentInfo, segmentSuffix, files);
+    BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Wed Nov  2 23:55:02 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs.m
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
@@ -30,8 +29,6 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.sep.IntStreamFactory;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.index.codecs.sep.IntIndexOutput;
-import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
-import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.intblock.VariableIntBlockIndexInput;
@@ -39,8 +36,6 @@ import org.apache.lucene.index.codecs.in
 import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.BlockTermsReader;
@@ -188,7 +183,7 @@ public class MockVariableIntBlockPosting
     PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
                                                               state.segmentInfo,
                                                               state.context,
-                                                              new MockIntFactory(baseBlockSize), state.formatId);
+                                                              new MockIntFactory(baseBlockSize), state.segmentSuffix);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;
@@ -198,7 +193,7 @@ public class MockVariableIntBlockPosting
                                                        state.segmentInfo.name,
                                                        state.termsIndexDivisor,
                                                        BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       state.formatId, state.context);
+                                                       state.segmentSuffix, state.context);
       success = true;
     } finally {
       if (!success) {
@@ -215,7 +210,7 @@ public class MockVariableIntBlockPosting
                                                 postingsReader,
                                                 state.context,
                                                 Lucene40PostingsFormat.TERMS_CACHE_SIZE,
-                                                state.formatId);
+                                                state.segmentSuffix);
       success = true;
       return ret;
     } finally {
@@ -230,9 +225,9 @@ public class MockVariableIntBlockPosting
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int formatId, Set<String> files) throws IOException {
-    SepPostingsReader.files(segmentInfo, formatId, files);
-    BlockTermsReader.files(dir, segmentInfo, formatId, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, formatId, files);
+  public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
+    SepPostingsReader.files(segmentInfo, segmentSuffix, files);
+    BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mockrandom/MockRandomPostingsFormat.java Wed Nov  2 23:55:02 2011
@@ -138,10 +138,10 @@ public class MockRandomPostingsFormat ex
     final long seed = seedRandom.nextLong();
 
     if (LuceneTestCase.VERBOSE) {
-      System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " formatID=" + state.formatId + " seed=" + seed);
+      System.out.println("MockRandomCodec: writing to seg=" + state.segmentName + " formatID=" + state.segmentSuffix + " seed=" + seed);
     }
 
-    final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.formatId, SEED_EXT);
+    final String seedFileName = IndexFileNames.segmentFileName(state.segmentName, state.segmentSuffix, SEED_EXT);
     final IndexOutput out = state.directory.createOutput(seedFileName, state.context);
     try {
       out.writeLong(seed);
@@ -272,11 +272,11 @@ public class MockRandomPostingsFormat ex
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
-    final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.formatId, SEED_EXT);
+    final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SEED_EXT);
     final IndexInput in = state.dir.openInput(seedFileName, state.context);
     final long seed = in.readLong();
     if (LuceneTestCase.VERBOSE) {
-      System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " formatID=" + state.formatId + " seed=" + seed);
+      System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " formatID=" + state.segmentSuffix + " seed=" + seed);
     }
     in.close();
 
@@ -294,12 +294,12 @@ public class MockRandomPostingsFormat ex
         System.out.println("MockRandomCodec: reading Sep postings");
       }
       postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
-                                             state.context, new MockIntStreamFactory(random), state.formatId);
+                                             state.context, new MockIntStreamFactory(random), state.segmentSuffix);
     } else {
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading Standard postings");
       }
-      postingsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
+      postingsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
     }
 
     if (random.nextBoolean()) {
@@ -325,7 +325,7 @@ public class MockRandomPostingsFormat ex
                                           state.segmentInfo.name,
                                           postingsReader,
                                           state.context,
-                                          state.formatId,
+                                          state.segmentSuffix,
                                           state.termsIndexDivisor);
         success = true;
       } finally {
@@ -359,7 +359,7 @@ public class MockRandomPostingsFormat ex
                                                      state.segmentInfo.name,
                                                      state.termsIndexDivisor,
                                                      BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                     state.formatId, state.context);
+                                                     state.segmentSuffix, state.context);
         } else {
           final int n2 = random.nextInt(3);
           if (n2 == 1) {
@@ -374,7 +374,7 @@ public class MockRandomPostingsFormat ex
                                                         state.fieldInfos,
                                                         state.segmentInfo.name,
                                                         state.termsIndexDivisor,
-                                                        state.formatId, state.context);
+                                                        state.segmentSuffix, state.context);
 
         }
 
@@ -396,7 +396,7 @@ public class MockRandomPostingsFormat ex
                                       postingsReader,
                                       state.context,
                                       termsCacheSize,
-                                      state.formatId);
+                                      state.segmentSuffix);
         success = true;
       } finally {
         if (!success) {
@@ -413,15 +413,15 @@ public class MockRandomPostingsFormat ex
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int codecId, Set<String> files) throws IOException {
-    final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SEED_EXT);    
+  public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
+    final String seedFileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, SEED_EXT);    
     files.add(seedFileName);
-    SepPostingsReader.files(segmentInfo, codecId, files);
-    Lucene40PostingsReader.files(dir, segmentInfo, codecId, files);
-    BlockTermsReader.files(dir, segmentInfo, codecId, files);
-    BlockTreeTermsReader.files(dir, segmentInfo, codecId, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
-    VariableGapTermsIndexReader.files(dir, segmentInfo, codecId, files);
+    SepPostingsReader.files(segmentInfo, segmentSuffix, files);
+    Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
+    BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
+    BlockTreeTermsReader.files(dir, segmentInfo, segmentSuffix, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
+    VariableGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
     // hackish!
     Iterator<String> it = files.iterator();
     while(it.hasNext()) {

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepDocValuesFormat.java Wed Nov  2 23:55:02 2011
@@ -48,7 +48,7 @@ public class MockSepDocValuesFormat exte
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo info, int formatId, Set<String> files) throws IOException {
-    SepDocValuesConsumer.files(dir, info, formatId, files);
+  public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+    SepDocValuesConsumer.files(dir, info, files);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/mocksep/MockSepPostingsFormat.java Wed Nov  2 23:55:02 2011
@@ -20,7 +20,6 @@ package org.apache.lucene.index.codecs.m
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.lucene.index.PerDocWriteState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
@@ -29,8 +28,6 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
 import org.apache.lucene.index.codecs.BlockTermsReader;
@@ -38,8 +35,6 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.lucene40.Lucene40PostingsFormat;
-import org.apache.lucene.index.codecs.sep.SepDocValuesConsumer;
-import org.apache.lucene.index.codecs.sep.SepDocValuesProducer;
 import org.apache.lucene.index.codecs.sep.SepPostingsWriter;
 import org.apache.lucene.index.codecs.sep.SepPostingsReader;
 import org.apache.lucene.store.Directory;
@@ -93,7 +88,7 @@ public class MockSepPostingsFormat exten
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
     PostingsReaderBase postingsReader = new SepPostingsReader(state.dir, state.segmentInfo,
-        state.context, new MockSingleIntFactory(), state.formatId);
+        state.context, new MockSingleIntFactory(), state.segmentSuffix);
 
     TermsIndexReaderBase indexReader;
     boolean success = false;
@@ -103,7 +98,7 @@ public class MockSepPostingsFormat exten
                                                        state.segmentInfo.name,
                                                        state.termsIndexDivisor,
                                                        BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       state.formatId, state.context);
+                                                       state.segmentSuffix, state.context);
       success = true;
     } finally {
       if (!success) {
@@ -120,7 +115,7 @@ public class MockSepPostingsFormat exten
                                                 postingsReader,
                                                 state.context,
                                                 Lucene40PostingsFormat.TERMS_CACHE_SIZE,
-                                                state.formatId);
+                                                state.segmentSuffix);
       success = true;
       return ret;
     } finally {
@@ -135,9 +130,9 @@ public class MockSepPostingsFormat exten
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int formatId, Set<String> files) throws IOException {
-    SepPostingsReader.files(segmentInfo, formatId, files);
-    BlockTermsReader.files(dir, segmentInfo, formatId, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, formatId, files);
+  public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
+    SepPostingsReader.files(segmentInfo, segmentSuffix, files);
+    BlockTermsReader.files(dir, segmentInfo, segmentSuffix, files);
+    FixedGapTermsIndexReader.files(dir, segmentInfo, segmentSuffix, files);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/nestedpulsing/NestedPulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/nestedpulsing/NestedPulsingPostingsFormat.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/nestedpulsing/NestedPulsingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/nestedpulsing/NestedPulsingPostingsFormat.java Wed Nov  2 23:55:02 2011
@@ -71,7 +71,7 @@ public class NestedPulsingPostingsFormat
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.formatId);
+    PostingsReaderBase docsReader = new Lucene40PostingsReader(state.dir, state.segmentInfo, state.context, state.segmentSuffix);
     PostingsReaderBase pulsingReaderInner = new PulsingPostingsReader(docsReader);
     PostingsReaderBase pulsingReader = new PulsingPostingsReader(pulsingReaderInner);
     boolean success = false;
@@ -80,7 +80,7 @@ public class NestedPulsingPostingsFormat
                                                     state.dir, state.fieldInfos, state.segmentInfo.name,
                                                     pulsingReader,
                                                     state.context,
-                                                    state.formatId,
+                                                    state.segmentSuffix,
                                                     state.termsIndexDivisor);
       success = true;
       return ret;
@@ -92,8 +92,8 @@ public class NestedPulsingPostingsFormat
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int id, Set<String> files) throws IOException {
-    Lucene40PostingsReader.files(dir, segmentInfo, id, files);
-    BlockTreeTermsReader.files(dir, segmentInfo, id, files);
+  public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) throws IOException {
+    Lucene40PostingsReader.files(dir, segmentInfo, segmentSuffix, files);
+    BlockTreeTermsReader.files(dir, segmentInfo, segmentSuffix, files);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java Wed Nov  2 23:55:02 2011
@@ -517,7 +517,7 @@ public class RAMOnlyPostingsFormat exten
     // nocommit -- ok to do this up front instead of
     // on close....?  should be ok?
     // Write our ID:
-    final String idFileName = IndexFileNames.segmentFileName(writeState.segmentName, writeState.formatId, ID_EXTENSION);
+    final String idFileName = IndexFileNames.segmentFileName(writeState.segmentName, writeState.segmentSuffix, ID_EXTENSION);
     IndexOutput out = writeState.directory.createOutput(idFileName, writeState.context);
     boolean success = false;
     try {
@@ -546,7 +546,7 @@ public class RAMOnlyPostingsFormat exten
     throws IOException {
 
     // Load our ID:
-    final String idFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.formatId, ID_EXTENSION);
+    final String idFileName = IndexFileNames.segmentFileName(readState.segmentInfo.name, readState.segmentSuffix, ID_EXTENSION);
     IndexInput in = readState.dir.openInput(idFileName, readState.context);
     boolean success = false;
     final int id;
@@ -568,8 +568,8 @@ public class RAMOnlyPostingsFormat exten
   }
 
   @Override
-  public void files(Directory dir, SegmentInfo segmentInfo, int formatId, Set<String> files) {
-    final String idFileName = IndexFileNames.segmentFileName(segmentInfo.name, formatId, ID_EXTENSION);
+  public void files(Directory dir, SegmentInfo segmentInfo, String segmentSuffix, Set<String> files) {
+    final String idFileName = IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, ID_EXTENSION);
     files.add(idFileName);
   }
 }

Modified: lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java?rev=1196874&r1=1196873&r2=1196874&view=diff
==============================================================================
--- lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java (original)
+++ lucene/dev/branches/lucene2621/lucene/src/test/org/apache/lucene/index/TestCompoundFile.java Wed Nov  2 23:55:02 2011
@@ -637,9 +637,9 @@ public class TestCompoundFile extends Lu
     CompoundFileDirectory csw = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), true);
     int size = 5 + random.nextInt(128);
     for (int j = 0; j < 2; j++) {
-      IndexOutput os = csw.createOutput("seg" + j + "_foo.txt", newIOContext(random));
+      IndexOutput os = csw.createOutput("seg_" + j + "_foo.txt", newIOContext(random));
       for (int i = 0; i < size; i++) {
-        os.writeInt(i);
+        os.writeInt(i*j);
       }
       os.close();
       String[] listAll = newDir.listAll();
@@ -654,10 +654,10 @@ public class TestCompoundFile extends Lu
     csw.close();
     CompoundFileDirectory csr = new CompoundFileDirectory(newDir, "d.cfs", newIOContext(random), false);
     for (int j = 0; j < 2; j++) {
-      IndexInput openInput = csr.openInput("seg" + j + "_foo.txt", newIOContext(random));
+      IndexInput openInput = csr.openInput("seg_" + j + "_foo.txt", newIOContext(random));
       assertEquals(size * 4, openInput.length());
       for (int i = 0; i < size; i++) {
-        assertEquals(i, openInput.readInt());
+        assertEquals(i*j, openInput.readInt());
       }
 
       openInput.close();