You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/05/09 15:19:39 UTC

svn commit: r1101016 [4/10] - in /lucene/dev/branches/bulkpostings: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/ dev-tools/maven/lucene/contrib/ant/ dev-tools/maven/lucene/contrib/db/bdb-je/ dev-tools/maven/luce...

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Mon May  9 13:19:28 2011
@@ -17,21 +17,22 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.Constants;
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
-import java.util.HashSet;
-import java.util.HashMap;
-import java.util.ArrayList;
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Constants;
 
 /**
  * Information about a segment such as it's name, directory, and files related
@@ -62,7 +63,7 @@ public final class SegmentInfo {
    * - NO says this field has no separate norms
    * >= YES says this field has separate norms with the specified generation
    */
-  private long[] normGen;                         
+  private Map<Integer,Long> normGen;
 
   private boolean isCompoundFile;         
 
@@ -83,6 +84,8 @@ public final class SegmentInfo {
   private boolean hasProx;                        // True if this segment has any fields with omitTermFreqAndPositions==false
 
   private boolean hasVectors;                     // True if this segment wrote term vectors
+  
+  private FieldInfos fieldInfos;
 
   private SegmentCodecs segmentCodecs;
 
@@ -100,7 +103,7 @@ public final class SegmentInfo {
   private long bufferedDeletesGen;
   
   public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
-                     boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) {
+                     boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors, FieldInfos fieldInfos) {
     this.name = name;
     this.docCount = docCount;
     this.dir = dir;
@@ -113,13 +116,14 @@ public final class SegmentInfo {
     this.hasVectors = hasVectors;
     delCount = 0;
     version = Constants.LUCENE_MAIN_VERSION;
+    this.fieldInfos = fieldInfos;
   }
 
   /**
    * Copy everything from src SegmentInfo into our instance.
    */
   void reset(SegmentInfo src) {
-    clearFiles();
+    clearFilesCache();
     version = src.version;
     name = src.name;
     docCount = src.docCount;
@@ -130,11 +134,14 @@ public final class SegmentInfo {
     docStoreIsCompoundFile = src.docStoreIsCompoundFile;
     hasVectors = src.hasVectors;
     hasProx = src.hasProx;
+    fieldInfos = src.fieldInfos == null ? null : (FieldInfos) src.fieldInfos.clone();
     if (src.normGen == null) {
       normGen = null;
     } else {
-      normGen = new long[src.normGen.length];
-      System.arraycopy(src.normGen, 0, normGen, 0, src.normGen.length);
+      normGen = new HashMap<Integer, Long>(src.normGen.size());
+      for (Entry<Integer,Long> entry : src.normGen.entrySet()) {
+        normGen.put(entry.getKey(), entry.getValue());
+      }
     }
     isCompoundFile = src.isCompoundFile;
     delCount = src.delCount;
@@ -184,9 +191,14 @@ public final class SegmentInfo {
     if (numNormGen == NO) {
       normGen = null;
     } else {
-      normGen = new long[numNormGen];
+      normGen = new HashMap<Integer, Long>();
       for(int j=0;j<numNormGen;j++) {
-        normGen[j] = input.readLong();
+        int fieldNumber = j;
+        if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+          fieldNumber = input.readInt();
+        }
+
+        normGen.put(fieldNumber, input.readLong());
       }
     }
     isCompoundFile = input.readByte() == YES;
@@ -197,13 +209,12 @@ public final class SegmentInfo {
     hasProx = input.readByte() == YES;
     
     // System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
-    segmentCodecs = new SegmentCodecs(codecs);
     if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
-      segmentCodecs.read(input);
+      segmentCodecs = new SegmentCodecs(codecs, input);
     } else {
       // codec ID on FieldInfo is 0 so it will simply use the first codec available
       // TODO what todo if preflex is not available in the provider? register it or fail?
-      segmentCodecs.codecs = new Codec[] { codecs.lookup("PreFlex")};
+      segmentCodecs = new SegmentCodecs(codecs, new Codec[] { codecs.lookup("PreFlex")});
     }
     diagnostics = input.readStringStringMap();
     
@@ -237,6 +248,24 @@ public final class SegmentInfo {
       }
     }
   }
+  
+  synchronized void loadFieldInfos(Directory dir, boolean checkCompoundFile) throws IOException {
+    if (fieldInfos == null) {
+      Directory dir0 = dir;
+      if (isCompoundFile && checkCompoundFile) {
+        dir0 = new CompoundFileReader(dir, IndexFileNames.segmentFileName(name,
+            "", IndexFileNames.COMPOUND_FILE_EXTENSION));
+      }
+      try {
+        fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
+            "", IndexFileNames.FIELD_INFOS_EXTENSION));
+      } finally {
+        if (dir != dir0) {
+          dir0.close();
+        }
+      }
+    }
+  }
 
   /**
    * Returns total size in bytes of all of files used by this segment (if
@@ -280,7 +309,12 @@ public final class SegmentInfo {
 
   public void setHasVectors(boolean v) {
     hasVectors = v;
-    clearFiles();
+    clearFilesCache();
+  }
+
+  public FieldInfos getFieldInfos() throws IOException {
+    loadFieldInfos(dir, true);
+    return fieldInfos;
   }
 
   public boolean hasDeletions() {
@@ -298,17 +332,18 @@ public final class SegmentInfo {
     } else {
       delGen++;
     }
-    clearFiles();
+    clearFilesCache();
   }
 
   void clearDelGen() {
     delGen = NO;
-    clearFiles();
+    clearFilesCache();
   }
 
   @Override
   public Object clone() {
-    SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, false);
+    final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, hasVectors,
+        fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
     si.docStoreOffset = docStoreOffset;
     si.docStoreSegment = docStoreSegment;
     si.docStoreIsCompoundFile = docStoreIsCompoundFile;
@@ -316,9 +351,11 @@ public final class SegmentInfo {
     si.delCount = delCount;
     si.diagnostics = new HashMap<String, String>(diagnostics);
     if (normGen != null) {
-      si.normGen = normGen.clone();
+      si.normGen = new HashMap<Integer, Long>();
+      for (Entry<Integer,Long> entry : normGen.entrySet()) {
+        si.normGen.put(entry.getKey(), entry.getValue());
+      }
     }
-    si.hasVectors = hasVectors;
     si.version = version;
     return si;
   }
@@ -339,7 +376,12 @@ public final class SegmentInfo {
    * @param fieldNumber the field index to check
    */
   public boolean hasSeparateNorms(int fieldNumber) {
-    return normGen != null && normGen[fieldNumber] != NO;
+    if (normGen == null) {
+      return false;
+    }
+
+    Long gen = normGen.get(fieldNumber);
+    return gen != null && gen.longValue() != NO;
   }
 
   /**
@@ -349,7 +391,7 @@ public final class SegmentInfo {
     if (normGen == null) {
       return false;
     } else {
-      for (long fieldNormGen : normGen) {
+      for (long fieldNormGen : normGen.values()) {
         if (fieldNormGen >= YES) {
           return true;
         }
@@ -359,10 +401,9 @@ public final class SegmentInfo {
     return false;
   }
 
-  void initNormGen(int numFields) {
+  void initNormGen() {
     if (normGen == null) { // normGen is null if this segments file hasn't had any norms set against it yet
-      normGen = new long[numFields];
-      Arrays.fill(normGen, NO);
+      normGen = new HashMap<Integer, Long>();
     }
   }
 
@@ -373,12 +414,13 @@ public final class SegmentInfo {
    * @param fieldIndex field whose norm file will be rewritten
    */
   void advanceNormGen(int fieldIndex) {
-    if (normGen[fieldIndex] == NO) {
-      normGen[fieldIndex] = YES;
+    Long gen = normGen.get(fieldIndex);
+    if (gen == null || gen.longValue() == NO) {
+      normGen.put(fieldIndex, new Long(YES));
     } else {
-      normGen[fieldIndex]++;
+      normGen.put(fieldIndex, gen+1);
     }
-    clearFiles();
+    clearFilesCache();
   }
 
   /**
@@ -388,7 +430,7 @@ public final class SegmentInfo {
    */
   public String getNormFileName(int number) {
     if (hasSeparateNorms(number)) {
-      return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen[number]);
+      return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen.get(number));
     } else {
       // single file for all norms 
       return IndexFileNames.fileNameFromGeneration(name, IndexFileNames.NORMS_EXTENSION, WITHOUT_GEN);
@@ -403,7 +445,7 @@ public final class SegmentInfo {
    */
   void setUseCompoundFile(boolean isCompoundFile) {
     this.isCompoundFile = isCompoundFile;
-    clearFiles();
+    clearFilesCache();
   }
 
   /**
@@ -433,7 +475,7 @@ public final class SegmentInfo {
   
   void setDocStoreIsCompoundFile(boolean v) {
     docStoreIsCompoundFile = v;
-    clearFiles();
+    clearFilesCache();
   }
   
   public String getDocStoreSegment() {
@@ -446,14 +488,14 @@ public final class SegmentInfo {
   
   void setDocStoreOffset(int offset) {
     docStoreOffset = offset;
-    clearFiles();
+    clearFilesCache();
   }
 
   void setDocStore(int offset, String segment, boolean isCompoundFile) {        
     docStoreOffset = offset;
     docStoreSegment = segment;
     docStoreIsCompoundFile = isCompoundFile;
-    clearFiles();
+    clearFilesCache();
   }
   
   /** Save this segment's info. */
@@ -474,9 +516,10 @@ public final class SegmentInfo {
     if (normGen == null) {
       output.writeInt(NO);
     } else {
-      output.writeInt(normGen.length);
-      for (long fieldNormGen : normGen) {
-        output.writeLong(fieldNormGen);
+      output.writeInt(normGen.size());
+      for (Entry<Integer,Long> entry : normGen.entrySet()) {
+        output.writeInt(entry.getKey());
+        output.writeLong(entry.getValue());
       }
     }
     
@@ -490,7 +533,7 @@ public final class SegmentInfo {
 
   void setHasProx(boolean hasProx) {
     this.hasProx = hasProx;
-    clearFiles();
+    clearFilesCache();
   }
 
   public boolean getHasProx() {
@@ -572,11 +615,11 @@ public final class SegmentInfo {
     }
 
     if (normGen != null) {
-      for (int i = 0; i < normGen.length; i++) {
-        long gen = normGen[i];
+      for (Entry<Integer,Long> entry : normGen.entrySet()) {
+        long gen = entry.getValue();
         if (gen >= YES) {
           // Definitely a separate norm file, with generation:
-          fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + i, gen));
+          fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + entry.getKey(), gen));
         }
       }
     }
@@ -588,7 +631,7 @@ public final class SegmentInfo {
 
   /* Called whenever any change is made that affects which
    * files this segment has. */
-  private void clearFiles() {
+  private void clearFilesCache() {
     files = null;
     sizeInBytesNoStore = -1;
     sizeInBytesWithStore = -1;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Mon May  9 13:19:28 2011
@@ -17,25 +17,27 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.NoSuchDirectoryException;
-import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.SegmentInfosReader;
-import org.apache.lucene.index.codecs.SegmentInfosWriter;
-import org.apache.lucene.util.ThreadInterruptedException;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.util.Vector;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Vector;
+
+import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
+import org.apache.lucene.index.codecs.SegmentInfosReader;
+import org.apache.lucene.index.codecs.SegmentInfosWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.NoSuchDirectoryException;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * A collection of segmentInfo objects with methods for operating on
@@ -64,6 +66,11 @@ public final class SegmentInfos extends 
    * starting with the current time in milliseconds forces to create unique version numbers.
    */
   public long version = System.currentTimeMillis();
+  
+  private long globalFieldMapVersion = 0; // version of the GFNM for the next commit
+  private long lastGlobalFieldMapVersion = 0; // version of the GFNM file we last successfully read or wrote
+  private long pendingMapVersion = -1; // version of the GFNM itself that we have last successfully written
+                                       // or -1 if we it was not written. This is set during prepareCommit 
 
   private long generation = 0;     // generation of the "segments_N" for the next commit
   private long lastGeneration = 0; // generation of the "segments_N" file we last successfully read
@@ -75,6 +82,8 @@ public final class SegmentInfos extends 
   private CodecProvider codecs;
 
   private int format;
+  
+  private FieldNumberBiMap globalFieldNumberMap; // this segments global field number map - lazy loaded on demand
 
   /**
    * If non-null, information about loading segments_N files
@@ -171,6 +180,15 @@ public final class SegmentInfos extends 
                                                  "",
                                                  lastGeneration);
   }
+  
+  private String getGlobalFieldNumberName(long version) {
+    /*
+     * This creates a file name ${version}.fnx without a leading underscore
+     * since this file might belong to more than one segment (global map) and
+     * could otherwise easily be confused with a per-segment file.
+     */
+    return IndexFileNames.segmentFileName(""+ version, "", IndexFileNames.GLOBAL_FIELD_NUM_MAP_EXTENSION);
+  }
 
   /**
    * Parse the generation off the segments file name and
@@ -261,6 +279,8 @@ public final class SegmentInfos extends 
         return null;
       }
     }.run();
+    // either we are on 4.0 or we don't have a lastGlobalFieldMapVersion i.e. its still set to 0
+    assert DefaultSegmentInfosWriter.FORMAT_4_0 <= format || (DefaultSegmentInfosWriter.FORMAT_4_0 > format && lastGlobalFieldMapVersion == 0); 
   }
 
   // Only non-null after prepareCommit has been called and
@@ -270,15 +290,24 @@ public final class SegmentInfos extends 
   private void write(Directory directory) throws IOException {
 
     String segmentFileName = getNextSegmentFileName();
-
+    final String globalFieldMapFile;
+    if (globalFieldNumberMap != null && globalFieldNumberMap.isDirty()) {
+      globalFieldMapFile = getGlobalFieldNumberName(++globalFieldMapVersion);
+      pendingMapVersion = writeGlobalFieldMap(globalFieldNumberMap, directory, globalFieldMapFile);
+    } else {
+      globalFieldMapFile = null;
+    }
+    
+    
     // Always advance the generation on write:
     if (generation == -1) {
       generation = 1;
     } else {
       generation++;
     }
-
+    
     IndexOutput segnOutput = null;
+    
 
     boolean success = false;
 
@@ -304,6 +333,16 @@ public final class SegmentInfos extends 
         } catch (Throwable t) {
           // Suppress so we keep throwing the original exception
         }
+        if (globalFieldMapFile != null) { // delete if written here
+          try {
+            // Try not to leave global field map in
+            // the index:
+            directory.deleteFile(globalFieldMapFile);
+          } catch (Throwable t) {
+            // Suppress so we keep throwing the original exception
+          }
+        }
+        pendingMapVersion = -1;
       }
     }
   }
@@ -719,6 +758,8 @@ public final class SegmentInfos extends 
   void updateGeneration(SegmentInfos other) {
     lastGeneration = other.lastGeneration;
     generation = other.generation;
+    lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
+    globalFieldMapVersion = other.globalFieldMapVersion;
   }
 
   final void rollbackCommit(Directory dir) throws IOException {
@@ -742,6 +783,16 @@ public final class SegmentInfos extends 
         // in our caller
       }
       pendingSegnOutput = null;
+      if (pendingMapVersion != -1) {
+        try {
+          final String fieldMapName = getGlobalFieldNumberName(globalFieldMapVersion--);
+          dir.deleteFile(fieldMapName);
+        } catch (Throwable t) {
+          // Suppress so we keep throwing the original exception
+          // in our caller
+        }
+        pendingMapVersion = -1;
+      }
     }
   }
 
@@ -760,6 +811,40 @@ public final class SegmentInfos extends 
       throw new IllegalStateException("prepareCommit was already called");
     write(dir);
   }
+  
+  private final long writeGlobalFieldMap(FieldNumberBiMap map, Directory dir, String name) throws IOException {
+    final IndexOutput output = dir.createOutput(name);
+    boolean success = false;
+    long version;
+    try {
+      version = map.write(output);
+      success = true;
+    } finally {
+      try {
+        output.close();
+      } catch (Throwable t) {
+        // throw orig excp
+      }
+      if (!success) {
+        try {
+          dir.deleteFile(name);
+        } catch (Throwable t) {
+          // throw orig excp
+        }
+      }
+    }
+    return version;
+  }
+  
+  private void readGlobalFieldMap(FieldNumberBiMap map, Directory dir) throws IOException {
+    final String name = getGlobalFieldNumberName(lastGlobalFieldMapVersion);
+    final IndexInput input = dir.openInput(name);
+    try {
+      map.read(input);
+    } finally {
+      input.close();
+    }
+  }
 
   /** Returns all file names referenced by SegmentInfo
    *  instances matching the provided Directory (ie files
@@ -769,7 +854,17 @@ public final class SegmentInfos extends 
   public Collection<String> files(Directory dir, boolean includeSegmentsFile) throws IOException {
     HashSet<String> files = new HashSet<String>();
     if (includeSegmentsFile) {
-      files.add(getCurrentSegmentFileName());
+      final String segmentFileName = getCurrentSegmentFileName();
+      if (segmentFileName != null) {
+        /*
+         * TODO: if lastGen == -1 we get might get null here it seems wrong to
+         * add null to the files set
+         */
+        files.add(segmentFileName);
+      }
+      if (lastGlobalFieldMapVersion > 0) {
+        files.add(getGlobalFieldNumberName(lastGlobalFieldMapVersion));
+      }
     }
     final int size = size();
     for(int i=0;i<size;i++) {
@@ -821,6 +916,17 @@ public final class SegmentInfos extends 
     }
 
     lastGeneration = generation;
+    if (pendingMapVersion != -1) {
+      /*
+       * TODO is it possible that the commit does not succeed here? if another
+       * commit happens at the same time and we lost the race between the
+       * prepareCommit and finishCommit the latest version is already
+       * incremented.
+       */
+      globalFieldNumberMap.commitLastVersion(pendingMapVersion);
+      pendingMapVersion = -1;
+      lastGlobalFieldMapVersion = globalFieldMapVersion;
+    }
 
     try {
       IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN);
@@ -848,6 +954,7 @@ public final class SegmentInfos extends 
     prepareCommit(dir);
     finishCommit(dir);
   }
+  
 
   public synchronized String toString(Directory directory) {
     StringBuilder buffer = new StringBuilder();
@@ -883,6 +990,8 @@ public final class SegmentInfos extends 
     clear();
     addAll(other);
     lastGeneration = other.lastGeneration;
+    lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
+    format = other.format;
   }
 
   /** Returns sum of all segment's docCounts.  Note that
@@ -900,4 +1009,49 @@ public final class SegmentInfos extends 
   public void changed() {
     version++;
   }
+  
+  /**
+   * Loads or returns the already loaded the global field number map for this {@link SegmentInfos}.
+   * If this {@link SegmentInfos} has no global field number map the returned instance is empty
+   */
+  synchronized FieldNumberBiMap getOrLoadGlobalFieldNumberMap(Directory dir) throws IOException {
+    if (globalFieldNumberMap != null) {
+      return globalFieldNumberMap;
+    }
+    final FieldNumberBiMap map  = new FieldNumberBiMap();
+    
+    if (lastGlobalFieldMapVersion > 0) {
+      // if we don't have a global map or this is a SI from a earlier version we just return the empty map;
+      readGlobalFieldMap(map, dir);
+    }
+    if (size() > 0) {
+      if (format > DefaultSegmentInfosWriter.FORMAT_4_0) {
+        assert lastGlobalFieldMapVersion == 0;
+        // build the map up if we open a pre 4.0 index
+        for (SegmentInfo info : this) {
+          final FieldInfos segFieldInfos = info.getFieldInfos();
+          for (FieldInfo fi : segFieldInfos) {
+            map.addOrGet(fi.name, fi.number);
+          }
+        }
+      }
+    }
+    return globalFieldNumberMap = map;
+  }
+
+  /**
+   * Called by {@link SegmentInfosReader} when reading the global field map version
+   */
+  public void setGlobalFieldMapVersion(long version) {
+    lastGlobalFieldMapVersion = globalFieldMapVersion = version;
+  }
+
+  public long getGlobalFieldMapVersion() {
+    return globalFieldMapVersion;
+  }
+  
+  // for testing
+  long getLastGlobalFieldMapVersion() {
+    return lastGlobalFieldMapVersion;
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon May  9 13:19:28 2011
@@ -26,16 +26,16 @@ import java.util.List;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.IndexReader.FieldOption;
 import org.apache.lucene.index.MergePolicy.MergeAbortedException;
-import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.MergeState;
+import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.MergeState;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.MultiBits;
+import org.apache.lucene.util.ReaderUtil;
 
 /**
  * The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
@@ -75,8 +75,8 @@ final class SegmentMerger {
     this.payloadProcessorProvider = payloadProcessorProvider;
     directory = dir;
     this.codecs = codecs;
-    this.fieldInfos = fieldInfos;
     segment = name;
+    this.fieldInfos = fieldInfos;
     if (merge != null) {
       checkAbort = new MergeState.CheckAbort(merge, directory);
     } else {
@@ -148,7 +148,7 @@ final class SegmentMerger {
       boolean storePayloads, boolean omitTFAndPositions)
       throws IOException {
     for (String field : names) {
-      fInfos.add(field, true, storeTermVectors,
+      fInfos.addOrUpdate(field, true, storeTermVectors,
           storePositionWithTermVector, storeOffsetWithTermVector, !reader
               .hasNorms(field), storePayloads, omitTFAndPositions);
     }
@@ -180,9 +180,8 @@ final class SegmentMerger {
         SegmentReader segmentReader = (SegmentReader) reader;
         boolean same = true;
         FieldInfos segmentFieldInfos = segmentReader.fieldInfos();
-        int numFieldInfos = segmentFieldInfos.size();
-        for (int j = 0; same && j < numFieldInfos; j++) {
-          same = fieldInfos.fieldName(j).equals(segmentFieldInfos.fieldName(j));
+        for (FieldInfo fi : segmentFieldInfos) {
+          same = fieldInfos.fieldName(fi.number).equals(fi.name);
         }
         if (same) {
           matchingSegmentReaders[i] = segmentReader;
@@ -208,9 +207,8 @@ final class SegmentMerger {
       if (reader instanceof SegmentReader) {
         SegmentReader segmentReader = (SegmentReader) reader;
         FieldInfos readerFieldInfos = segmentReader.fieldInfos();
-        int numReaderFieldInfos = readerFieldInfos.size();
-        for (int j = 0; j < numReaderFieldInfos; j++) {
-          fieldInfos.add(readerFieldInfos.fieldInfo(j));
+        for (FieldInfo fi : readerFieldInfos) {
+          fieldInfos.add(fi);
         }
       } else {
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false);
@@ -220,17 +218,17 @@ final class SegmentMerger {
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS), false, false, false, false, true);
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, false);
         addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, false);
-        fieldInfos.add(reader.getFieldNames(FieldOption.UNINDEXED), false);
+        fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
       }
     }
-    final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, this.codecs);
-    fieldInfos.write(directory, segment + ".fnm");
+    final SegmentCodecs codecInfo = fieldInfos.buildSegmentCodecs(false);
+    fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
 
     int docCount = 0;
 
     setMatchingSegmentReaders();
 
-    final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
+    final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment);
 
     try {
       int idx = 0;
@@ -312,7 +310,7 @@ final class SegmentMerger {
         // NOTE: it's very important to first assign to doc then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         Document doc = reader.document(j);
-        fieldsWriter.addDocument(doc);
+        fieldsWriter.addDocument(doc, fieldInfos);
         docCount++;
         checkAbort.work(300);
       }
@@ -339,7 +337,7 @@ final class SegmentMerger {
         // NOTE: it's very important to first assign to doc then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         Document doc = reader.document(docCount);
-        fieldsWriter.addDocument(doc);
+        fieldsWriter.addDocument(doc, fieldInfos);
         checkAbort.work(300);
       }
     }
@@ -579,8 +577,7 @@ final class SegmentMerger {
   private void mergeNorms() throws IOException {
     IndexOutput output = null;
     try {
-      for (int i = 0, numFieldInfos = fieldInfos.size(); i < numFieldInfos; i++) {
-        final FieldInfo fi = fieldInfos.fieldInfo(i);
+      for (FieldInfo fi : fieldInfos) {
         if (fi.isIndexed && !fi.omitNorms) {
           if (output == null) { 
             output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION));

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon May  9 13:19:28 2011
@@ -22,23 +22,22 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
-
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.CloseableThreadLocal;
-import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CloseableThreadLocal;
 
 /**
  * @lucene.experimental
@@ -119,8 +118,8 @@ public class SegmentReader extends Index
           dir0 = cfsReader;
         }
         cfsDir = dir0;
-
-        fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+        si.loadFieldInfos(cfsDir, false); // prevent opening the CFS to load fieldInfos
+        fieldInfos = si.getFieldInfos();
         
         this.termsIndexDivisor = termsIndexDivisor;
         
@@ -598,12 +597,12 @@ public class SegmentReader extends Index
                                   && (!si.hasDeletions() || this.si.getDelFileName().equals(si.getDelFileName()));
     boolean normsUpToDate = true;
     
-    boolean[] fieldNormsChanged = new boolean[core.fieldInfos.size()];
-    final int fieldCount = core.fieldInfos.size();
-    for (int i = 0; i < fieldCount; i++) {
-      if (!this.si.getNormFileName(i).equals(si.getNormFileName(i))) {
+    Set<Integer> fieldNormsChanged = new HashSet<Integer>();
+    for (FieldInfo fi : core.fieldInfos) {
+      int fieldNumber = fi.number;
+      if (!this.si.getNormFileName(fieldNumber).equals(si.getNormFileName(fieldNumber))) {
         normsUpToDate = false;
-        fieldNormsChanged[i] = true;
+        fieldNormsChanged.add(fieldNumber);
       }
     }
 
@@ -659,11 +658,10 @@ public class SegmentReader extends Index
       clone.norms = new HashMap<String,Norm>();
 
       // Clone norms
-      for (int i = 0; i < fieldNormsChanged.length; i++) {
-
+      for (FieldInfo fi : core.fieldInfos) {
         // Clone unchanged norms to the cloned reader
-        if (doClone || !fieldNormsChanged[i]) {
-          final String curField = core.fieldInfos.fieldInfo(i).name;
+        if (doClone || !fieldNormsChanged.contains(fi.number)) {
+          final String curField = fi.name;
           Norm norm = this.norms.get(curField);
           if (norm != null)
             clone.norms.put(curField, (Norm) norm.clone());
@@ -735,7 +733,7 @@ public class SegmentReader extends Index
     }
 
     if (normsDirty) {               // re-write norms
-      si.initNormGen(core.fieldInfos.size());
+      si.initNormGen();
       for (final Norm norm : norms.values()) {
         if (norm.dirty) {
           norm.reWrite(si);
@@ -880,8 +878,7 @@ public class SegmentReader extends Index
     ensureOpen();
 
     Set<String> fieldSet = new HashSet<String>();
-    for (int i = 0; i < core.fieldInfos.size(); i++) {
-      FieldInfo fi = core.fieldInfos.fieldInfo(i);
+    for (FieldInfo fi : core.fieldInfos) {
       if (fieldOption == IndexReader.FieldOption.ALL) {
         fieldSet.add(fi.name);
       }
@@ -954,8 +951,7 @@ public class SegmentReader extends Index
   private void openNorms(Directory cfsDir, int readBufferSize) throws IOException {
     long nextNormSeek = SegmentMerger.NORMS_HEADER.length; //skip header (header unused for now)
     int maxDoc = maxDoc();
-    for (int i = 0; i < core.fieldInfos.size(); i++) {
-      FieldInfo fi = core.fieldInfos.fieldInfo(i);
+    for (FieldInfo fi : core.fieldInfos) {
       if (norms.containsKey(fi.name)) {
         // in case this SegmentReader is being re-opened, we might be able to
         // reuse some norm instances and skip loading them here

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon May  9 13:19:28 2011
@@ -27,15 +27,13 @@ final class StoredFieldsWriter {
 
   FieldsWriter fieldsWriter;
   final DocumentsWriter docWriter;
-  final FieldInfos fieldInfos;
   int lastDocID;
 
   PerDoc[] docFreeList = new PerDoc[1];
   int freeCount;
 
-  public StoredFieldsWriter(DocumentsWriter docWriter, FieldInfos fieldInfos) {
+  public StoredFieldsWriter(DocumentsWriter docWriter) {
     this.docWriter = docWriter;
-    this.fieldInfos = fieldInfos;
   }
 
   public StoredFieldsWriterPerThread addThread(DocumentsWriter.DocState docState) throws IOException {
@@ -62,7 +60,7 @@ final class StoredFieldsWriter {
 
   private synchronized void initFieldsWriter() throws IOException {
     if (fieldsWriter == null) {
-      fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos);
+      fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment());
       lastDocID = 0;
     }
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java Mon May  9 13:19:28 2011
@@ -32,7 +32,7 @@ final class StoredFieldsWriterPerThread 
   public StoredFieldsWriterPerThread(DocumentsWriter.DocState docState, StoredFieldsWriter storedFieldsWriter) throws IOException {
     this.storedFieldsWriter = storedFieldsWriter;
     this.docState = docState;
-    localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, storedFieldsWriter.fieldInfos);
+    localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null);
   }
 
   public void startDocument() {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon May  9 13:19:28 2011
@@ -57,12 +57,6 @@ final class TermsHash extends InvertedDo
   }
 
   @Override
-  void setFieldInfos(FieldInfos fieldInfos) {
-    this.fieldInfos = fieldInfos;
-    consumer.setFieldInfos(fieldInfos);
-  }
-
-  @Override
   public void abort() {
     consumer.abort();
     if (nextTermsHash != null)

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Mon May  9 13:19:28 2011
@@ -25,10 +25,4 @@ abstract class TermsHashConsumer {
   abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
   abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
   abstract void abort();
-
-  FieldInfos fieldInfos;
-
-  void setFieldInfos(FieldInfos fieldInfos) {
-    this.fieldInfos = fieldInfos;
   }
-}

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Mon May  9 13:19:28 2011
@@ -39,6 +39,7 @@ final class TermsHashPerField extends In
   final DocumentsWriter.DocState docState;
   final FieldInvertState fieldState;
   TermToBytesRefAttribute termAtt;
+  BytesRef termBytesRef;
 
   // Copied from our perThread
   final IntBlockPool intPool;
@@ -53,7 +54,6 @@ final class TermsHashPerField extends In
   final BytesRefHash bytesHash;
  
   ParallelPostingsArray postingsArray;
-  private final BytesRef termBytesRef;
   private final AtomicLong bytesUsed;
 
   public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHashPerThread perThread, final TermsHashPerThread nextPerThread, final FieldInfo fieldInfo) {
@@ -70,7 +70,6 @@ final class TermsHashPerField extends In
     bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts); 
     streamCount = consumer.getStreamCount();
     numPostingInt = 2*streamCount;
-    termBytesRef = perThread.termBytesRef;
     this.fieldInfo = fieldInfo;
     if (nextPerThread != null)
       nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo);
@@ -119,6 +118,7 @@ final class TermsHashPerField extends In
   @Override
   void start(Fieldable f) {
     termAtt = fieldState.attributeSource.getAttribute(TermToBytesRefAttribute.class);
+    termBytesRef = termAtt.getBytesRef();
     consumer.start(f);
     if (nextPerField != null) {
       nextPerField.start(f);
@@ -181,7 +181,7 @@ final class TermsHashPerField extends In
     // Get the text & hash of this term.
     int termID;
     try{
-       termID = bytesHash.add(termBytesRef, termAtt.toBytesRef(termBytesRef));
+       termID = bytesHash.add(termBytesRef, termAtt.fillBytesRef());
     }catch (MaxBytesLengthExceededException e) {
       // Not enough room in current block
       // Just skip this term, to remain as robust as

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java Mon May  9 13:19:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.util.ByteBlockPool;
-import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
 
@@ -35,8 +34,6 @@ final class TermsHashPerThread extends I
   
   final boolean primary;
   final DocumentsWriter.DocState docState;
-  // Used by perField to obtain terms from the analysis chain
-  final BytesRef termBytesRef = new BytesRef(10);
 
   public TermsHashPerThread(DocInverterPerThread docInverterPerThread, final TermsHash termsHash, final TermsHash nextTermsHash, final TermsHashPerThread primaryPerThread) {
     docState = docInverterPerThread.docState;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Mon May  9 13:19:28 2011
@@ -56,7 +56,9 @@ public class DefaultSegmentInfosReader e
   
       infos.version = input.readLong(); // read version
       infos.counter = input.readInt(); // read counter
-  
+      if (infos.getFormat() <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+        infos.setGlobalFieldMapVersion(input.readLong());
+      }
       for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
         SegmentInfo si = new SegmentInfo(directory, format, input, codecs);
         if (si.getVersion() == null) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Mon May  9 13:19:28 2011
@@ -59,6 +59,7 @@ public class DefaultSegmentInfosWriter e
     out.writeInt(FORMAT_CURRENT); // write FORMAT
     out.writeLong(infos.version);
     out.writeInt(infos.counter); // write counter
+    out.writeLong(infos.getGlobalFieldMapVersion());
     out.writeInt(infos.size()); // write infos
     for (SegmentInfo si : infos) {
       si.write(out);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Mon May  9 13:19:28 2011
@@ -19,15 +19,16 @@ package org.apache.lucene.index.codecs.p
 
 import java.io.IOException;
 import java.util.Collection;
-import java.util.Iterator;
-import java.util.TreeMap;
+import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
-import java.util.Comparator;
+import java.util.TreeMap;
 
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.BulkPostingsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldsEnum;
@@ -36,7 +37,6 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -96,13 +96,11 @@ public class PreFlexFields extends Field
       // so that if an index update removes them we'll still have them
       freqStream = dir.openInput(info.name + ".frq", readBufferSize);
       boolean anyProx = false;
-      final int numFields = fieldInfos.size();
-      for(int i=0;i<numFields;i++) {
-        final FieldInfo fieldInfo = fieldInfos.fieldInfo(i);
-        if (fieldInfo.isIndexed) {
-          fields.put(fieldInfo.name, fieldInfo);
-          preTerms.put(fieldInfo.name, new PreTerms(fieldInfo));
-          if (!fieldInfo.omitTermFreqAndPositions) {
+      for (FieldInfo fi : fieldInfos) {
+        if (fi.isIndexed) {
+          fields.put(fi.name, fi);
+          preTerms.put(fi.name, new PreTerms(fi));
+          if (!fi.omitTermFreqAndPositions) {
             anyProx = true;
           }
         }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java Mon May  9 13:19:28 2011
@@ -532,18 +532,19 @@ public abstract class QueryParserBase {
       // ignore
     }
 
+    BytesRef bytes = termAtt == null ? null : termAtt.getBytesRef();
+
     if (numTokens == 0)
       return null;
     else if (numTokens == 1) {
-      BytesRef term = new BytesRef();
       try {
         boolean hasNext = buffer.incrementToken();
         assert hasNext == true;
-        termAtt.toBytesRef(term);
+        termAtt.fillBytesRef();
       } catch (IOException e) {
         // safe to ignore, because we know the number of tokens
       }
-      return newTermQuery(new Term(field, term));
+      return newTermQuery(new Term(field, new BytesRef(bytes)));
     } else {
       if (severalTokensAtSamePosition || (!quoted && !autoGeneratePhraseQueries)) {
         if (positionCount == 1 || (!quoted && !autoGeneratePhraseQueries)) {
@@ -554,17 +555,15 @@ public abstract class QueryParserBase {
             BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD;
 
           for (int i = 0; i < numTokens; i++) {
-            BytesRef term = new BytesRef();
             try {
               boolean hasNext = buffer.incrementToken();
               assert hasNext == true;
-              termAtt.toBytesRef(term);
+              termAtt.fillBytesRef();
             } catch (IOException e) {
               // safe to ignore, because we know the number of tokens
             }
-
             Query currentQuery = newTermQuery(
-                new Term(field, term));
+                new Term(field, new BytesRef(bytes)));
             q.add(currentQuery, occur);
           }
           return q;
@@ -576,12 +575,11 @@ public abstract class QueryParserBase {
           List<Term> multiTerms = new ArrayList<Term>();
           int position = -1;
           for (int i = 0; i < numTokens; i++) {
-            BytesRef term = new BytesRef();
             int positionIncrement = 1;
             try {
               boolean hasNext = buffer.incrementToken();
               assert hasNext == true;
-              termAtt.toBytesRef(term);
+              termAtt.fillBytesRef();
               if (posIncrAtt != null) {
                 positionIncrement = posIncrAtt.getPositionIncrement();
               }
@@ -598,7 +596,7 @@ public abstract class QueryParserBase {
               multiTerms.clear();
             }
             position += positionIncrement;
-            multiTerms.add(new Term(field, term));
+            multiTerms.add(new Term(field, new BytesRef(bytes)));
           }
           if (enablePositionIncrements) {
             mpq.add(multiTerms.toArray(new Term[0]),position);
@@ -613,15 +611,13 @@ public abstract class QueryParserBase {
         pq.setSlop(phraseSlop);
         int position = -1;
 
-
         for (int i = 0; i < numTokens; i++) {
-          BytesRef term = new BytesRef();
           int positionIncrement = 1;
 
           try {
             boolean hasNext = buffer.incrementToken();
             assert hasNext == true;
-            termAtt.toBytesRef(term);
+            termAtt.fillBytesRef();
             if (posIncrAtt != null) {
               positionIncrement = posIncrAtt.getPositionIncrement();
             }
@@ -631,9 +627,9 @@ public abstract class QueryParserBase {
 
           if (enablePositionIncrements) {
             position += positionIncrement;
-            pq.add(new Term(field, term),position);
+            pq.add(new Term(field, new BytesRef(bytes)),position);
           } else {
-            pq.add(new Term(field, term));
+            pq.add(new Term(field, new BytesRef(bytes)));
           }
         }
         return pq;
@@ -796,13 +792,13 @@ public abstract class QueryParserBase {
       source = analyzer.tokenStream(field, new StringReader(part));
     }
       
-    BytesRef result = new BytesRef();
     TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
-      
+    BytesRef bytes = termAtt.getBytesRef();
+
     try {
       if (!source.incrementToken())
         throw new IllegalArgumentException("analyzer returned no terms for range part: " + part);
-      termAtt.toBytesRef(result);
+      termAtt.fillBytesRef();
       if (source.incrementToken())
         throw new IllegalArgumentException("analyzer returned too many terms for range part: " + part);
     } catch (IOException e) {
@@ -812,8 +808,8 @@ public abstract class QueryParserBase {
     try {
       source.close();
     } catch (IOException ignored) {}
-      
-    return result;
+    
+    return new BytesRef(bytes);
   }
 
   /**

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java Mon May  9 13:19:28 2011
@@ -20,7 +20,7 @@ import org.apache.lucene.index.FieldInve
  */
 
 /** Expert: Default scoring implementation. */
-public class DefaultSimilarity extends Similarity implements SimilarityProvider {
+public class DefaultSimilarity extends Similarity {
 
   /** Implemented as
    *  <code>state.getBoost()*lengthNorm(numTerms)</code>, where
@@ -40,11 +40,6 @@ public class DefaultSimilarity extends S
     return state.getBoost() * ((float) (1.0 / Math.sqrt(numTerms)));
   }
   
-  /** Implemented as <code>1/sqrt(sumOfSquaredWeights)</code>. */
-  public float queryNorm(float sumOfSquaredWeights) {
-    return (float)(1.0 / Math.sqrt(sumOfSquaredWeights));
-  }
-
   /** Implemented as <code>sqrt(freq)</code>. */
   @Override
   public float tf(float freq) {
@@ -63,11 +58,6 @@ public class DefaultSimilarity extends S
     return (float)(Math.log(numDocs/(double)(docFreq+1)) + 1.0);
   }
     
-  /** Implemented as <code>overlap / maxOverlap</code>. */
-  public float coord(int overlap, int maxOverlap) {
-    return overlap / (float)maxOverlap;
-  }
-
   // Default true
   protected boolean discountOverlaps = true;
 
@@ -88,12 +78,4 @@ public class DefaultSimilarity extends S
   public boolean getDiscountOverlaps() {
     return discountOverlaps;
   }
-
-  /** 
-   * Returns this default implementation for all fields.
-   * Override this method to customize scoring on a per-field basis.
-   */
-  public Similarity get(String field) {
-    return this;
-  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon May  9 13:19:28 2011
@@ -72,7 +72,7 @@ public class IndexSearcher {
   private final ExecutorService executor;
 
   // the default SimilarityProvider
-  private static final SimilarityProvider defaultProvider = new DefaultSimilarity();
+  private static final SimilarityProvider defaultProvider = new DefaultSimilarityProvider();
   
   /**
    * Expert: returns a default SimilarityProvider instance.

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java Mon May  9 13:19:28 2011
@@ -66,10 +66,10 @@ public class QueryTermVector implements 
           final TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
 
           hasMoreTokens = stream.incrementToken();
+          BytesRef bytes = termAtt.getBytesRef();
           while (hasMoreTokens) {
-            BytesRef bytes = new BytesRef();
-            termAtt.toBytesRef(bytes);
-            terms.add(bytes);
+            termAtt.fillBytesRef();
+            terms.add(new BytesRef(bytes));
             hasMoreTokens = stream.incrementToken();
           }
           processTerms(terms.toArray(new BytesRef[terms.size()]));

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java Mon May  9 13:19:28 2011
@@ -362,7 +362,7 @@ import org.apache.lucene.util.SmallFloat
  *      than another document with fewer query terms.
  *      This is a search time factor computed in
  *      {@link SimilarityProvider#coord(int, int) coord(q,d)}
- *      by the Similarity in effect at search time.
+ *      by the SimilarityProvider in effect at search time.
  *      <br>&nbsp;<br>
  *    </li>
  *
@@ -373,17 +373,17 @@ import org.apache.lucene.util.SmallFloat
  *      is a normalizing factor used to make scores between queries comparable.
  *      This factor does not affect document ranking (since all ranked documents are multiplied by the same factor),
  *      but rather just attempts to make scores from different queries (or even different indexes) comparable.
- *      This is a search time factor computed by the Similarity in effect at search time.
+ *      This is a search time factor computed by the SimilarityProvider in effect at search time.
  *
  *      The default computation in
- *      {@link org.apache.lucene.search.DefaultSimilarity#queryNorm(float) DefaultSimilarity}
+ *      {@link org.apache.lucene.search.DefaultSimilarityProvider#queryNorm(float) DefaultSimilarityProvider}
  *      produces a <a href="http://en.wikipedia.org/wiki/Euclidean_norm#Euclidean_norm">Euclidean norm</a>:
  *      <br>&nbsp;<br>
  *      <table cellpadding="1" cellspacing="0" border="0" align="center">
  *        <tr>
  *          <td valign="middle" align="right" rowspan="1">
  *            queryNorm(q)  &nbsp; = &nbsp;
- *            {@link org.apache.lucene.search.DefaultSimilarity#queryNorm(float) queryNorm(sumOfSquaredWeights)}
+ *            {@link org.apache.lucene.search.DefaultSimilarityProvider#queryNorm(float) queryNorm(sumOfSquaredWeights)}
  *            &nbsp; = &nbsp;
  *          </td>
  *          <td valign="middle" align="center" rowspan="1">

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java Mon May  9 13:19:28 2011
@@ -80,6 +80,9 @@ public abstract class DataInput implemen
    * @see DataOutput#writeVInt(int)
    */
   public int readVInt() throws IOException {
+    /* This is the original code of this method,
+     * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
+     * readByte() is inlined. So the loop was unwinded!
     byte b = readByte();
     int i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@@ -87,6 +90,22 @@ public abstract class DataInput implemen
       i |= (b & 0x7F) << shift;
     }
     return i;
+    */
+    byte b = readByte();
+    int i = b & 0x7F;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7F) << 7;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7F) << 14;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7F) << 21;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    assert (b & 0x80) == 0;
+    return i | ((b & 0x7F) << 28);
   }
 
   /** Reads eight bytes and returns a long.
@@ -100,6 +119,9 @@ public abstract class DataInput implemen
    * nine bytes.  Smaller values take fewer bytes.  Negative numbers are not
    * supported. */
   public long readVLong() throws IOException {
+    /* This is the original code of this method,
+     * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
+     * readByte() is inlined. So the loop was unwinded!
     byte b = readByte();
     long i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@@ -107,6 +129,34 @@ public abstract class DataInput implemen
       i |= (b & 0x7FL) << shift;
     }
     return i;
+    */
+    byte b = readByte();
+    long i = b & 0x7FL;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 7;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 14;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 21;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 28;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 35;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 42;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    i |= (b & 0x7FL) << 49;
+    if ((b & 0x80) == 0) return i;
+    b = readByte();
+    assert (b & 0x80) == 0;
+    return i | ((b & 0x7FL) << 56);
   }
 
   /** Reads a string.

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Mon May  9 13:19:28 2011
@@ -563,6 +563,7 @@ public final class ArrayUtil {
    * @param toIndex end index (exclusive)
    */
   public static <T> void quickSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+    if (toIndex-fromIndex <= 1) return;
     getSorter(a, comp).quickSort(fromIndex, toIndex-1);
   }
   
@@ -581,6 +582,7 @@ public final class ArrayUtil {
    * @param toIndex end index (exclusive)
    */
   public static <T extends Comparable<? super T>> void quickSort(T[] a, int fromIndex, int toIndex) {
+    if (toIndex-fromIndex <= 1) return;
     getSorter(a).quickSort(fromIndex, toIndex-1);
   }
   
@@ -601,6 +603,7 @@ public final class ArrayUtil {
    * @param toIndex end index (exclusive)
    */
   public static <T> void mergeSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+    if (toIndex-fromIndex <= 1) return;
     getSorter(a, comp).mergeSort(fromIndex, toIndex-1);
   }
   
@@ -619,6 +622,7 @@ public final class ArrayUtil {
    * @param toIndex end index (exclusive)
    */
   public static <T extends Comparable<? super T>> void mergeSort(T[] a, int fromIndex, int toIndex) {
+    if (toIndex-fromIndex <= 1) return;
     getSorter(a).mergeSort(fromIndex, toIndex-1);
   }
   
@@ -639,6 +643,7 @@ public final class ArrayUtil {
    * @param toIndex end index (exclusive)
    */
   public static <T> void insertionSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+    if (toIndex-fromIndex <= 1) return;
     getSorter(a, comp).insertionSort(fromIndex, toIndex-1);
   }
   
@@ -657,6 +662,7 @@ public final class ArrayUtil {
    * @param toIndex end index (exclusive)
    */
   public static <T extends Comparable<? super T>> void insertionSort(T[] a, int fromIndex, int toIndex) {
+    if (toIndex-fromIndex <= 1) return;
     getSorter(a).insertionSort(fromIndex, toIndex-1);
   }
   

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java Mon May  9 13:19:28 2011
@@ -100,7 +100,9 @@ public final class CollectionUtil {
    * @throws IllegalArgumentException if list is e.g. a linked list without random access.
    */
   public static <T> void quickSort(List<T> list, Comparator<? super T> comp) {
-    getSorter(list, comp).quickSort(0, list.size()-1);
+    final int size = list.size();
+    if (size <= 1) return;
+    getSorter(list, comp).quickSort(0, size-1);
   }
   
   /**
@@ -110,7 +112,9 @@ public final class CollectionUtil {
    * @throws IllegalArgumentException if list is e.g. a linked list without random access.
    */
   public static <T extends Comparable<? super T>> void quickSort(List<T> list) {
-    getSorter(list).quickSort(0, list.size()-1);
+    final int size = list.size();
+    if (size <= 1) return;
+    getSorter(list).quickSort(0, size-1);
   }
 
   // mergeSorts:
@@ -122,7 +126,9 @@ public final class CollectionUtil {
    * @throws IllegalArgumentException if list is e.g. a linked list without random access.
    */
   public static <T> void mergeSort(List<T> list, Comparator<? super T> comp) {
-    getSorter(list, comp).mergeSort(0, list.size()-1);
+    final int size = list.size();
+    if (size <= 1) return;
+    getSorter(list, comp).mergeSort(0, size-1);
   }
   
   /**
@@ -132,7 +138,9 @@ public final class CollectionUtil {
    * @throws IllegalArgumentException if list is e.g. a linked list without random access.
    */
   public static <T extends Comparable<? super T>> void mergeSort(List<T> list) {
-    getSorter(list).mergeSort(0, list.size()-1);
+    final int size = list.size();
+    if (size <= 1) return;
+    getSorter(list).mergeSort(0, size-1);
   }
 
   // insertionSorts:
@@ -144,7 +152,9 @@ public final class CollectionUtil {
    * @throws IllegalArgumentException if list is e.g. a linked list without random access.
    */
   public static <T> void insertionSort(List<T> list, Comparator<? super T> comp) {
-    getSorter(list, comp).insertionSort(0, list.size()-1);
+    final int size = list.size();
+    if (size <= 1) return;
+    getSorter(list, comp).insertionSort(0, size-1);
   }
   
   /**
@@ -154,7 +164,9 @@ public final class CollectionUtil {
    * @throws IllegalArgumentException if list is e.g. a linked list without random access.
    */
   public static <T extends Comparable<? super T>> void insertionSort(List<T> list) {
-    getSorter(list).insertionSort(0, list.size()-1);
+    final int size = list.size();
+    if (size <= 1) return;
+    getSorter(list).insertionSort(0, size-1);
   }
   
 }
\ No newline at end of file

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java Mon May  9 13:19:28 2011
@@ -21,8 +21,10 @@ package org.apache.lucene.util;
  * least element can always be found in constant time.  Put()'s and pop()'s
  * require log(size) time.
  *
- * <p><b>NOTE</b>: This class pre-allocates a full array of
- * length <code>maxSize+1</code>, in {@link #initialize}.
+ * <p><b>NOTE</b>: This class will pre-allocate a full array of
+ * length <code>maxSize+1</code> if instantiated via the
+ * {@link #PriorityQueue(int,boolean)} constructor with
+ * <code>prepopulate</code> set to <code>true</code>.
  * 
  * @lucene.internal
 */
@@ -83,9 +85,10 @@ public abstract class PriorityQueue<T> {
 
   /**
    * This method can be overridden by extending classes to return a sentinel
-   * object which will be used by {@link #initialize(int)} to fill the queue, so
-   * that the code which uses that queue can always assume it's full and only
-   * change the top without attempting to insert any new object.<br>
+   * object which will be used by the {@link PriorityQueue#PriorityQueue(int,boolean)} 
+   * constructor to fill the queue, so that the code which uses that queue can always
+   * assume it's full and only change the top without attempting to insert any new
+   * object.<br>
    * 
    * Those sentinel values should always compare worse than any non-sentinel
    * value (i.e., {@link #lessThan} should always favor the
@@ -111,11 +114,11 @@ public abstract class PriorityQueue<T> {
    * </pre>
    * 
    * <b>NOTE:</b> if this method returns a non-null value, it will be called by
-   * {@link #initialize(int)} {@link #size()} times, relying on a new object to
-   * be returned and will not check if it's null again. Therefore you should
-   * ensure any call to this method creates a new instance and behaves
-   * consistently, e.g., it cannot return null if it previously returned
-   * non-null.
+   * the {@link PriorityQueue#PriorityQueue(int,boolean)} constructor 
+   * {@link #size()} times, relying on a new object to be returned and will not
+   * check if it's null again. Therefore you should ensure any call to this
+   * method creates a new instance and behaves consistently, e.g., it cannot
+   * return null if it previously returned non-null.
    * 
    * @return the sentinel object to use to pre-populate the queue, or null if
    *         sentinel objects are not supported.

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java Mon May  9 13:19:28 2011
@@ -29,6 +29,7 @@
 
 package org.apache.lucene.util.automaton;
 
+import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
@@ -65,7 +66,7 @@ final public class SpecialOperations {
    */
   public static boolean isFinite(Automaton a) {
     if (a.isSingleton()) return true;
-    return isFinite(a.initial, new HashSet<State>());
+    return isFinite(a.initial, new BitSet(a.getNumberOfStates()), new BitSet(a.getNumberOfStates()));
   }
   
   /**
@@ -74,11 +75,12 @@ final public class SpecialOperations {
    */
   // TODO: not great that this is recursive... in theory a
   // large automata could exceed java's stack
-  private static boolean isFinite(State s, HashSet<State> path) {
-    path.add(s);
+  private static boolean isFinite(State s, BitSet path, BitSet visited) {
+    path.set(s.number);
     for (Transition t : s.getTransitions())
-      if (path.contains(t.to) || !isFinite(t.to, path)) return false;
-    path.remove(s);
+      if (path.get(t.to.number) || (!visited.get(t.to.number) && !isFinite(t.to, path, visited))) return false;
+    path.clear(s.number);
+    visited.set(s.number);
     return true;
   }
   

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon May  9 13:19:28 2011
@@ -150,10 +150,24 @@ public class RandomIndexWriter implement
     return getReader(true);
   }
 
+  private void doRandomOptimize() throws IOException {
+    final int segCount = w.getSegmentCount();
+    if (r.nextBoolean() || segCount == 0) {
+      // full optimize
+      w.optimize();
+    } else {
+      // partial optimize
+      final int limit = _TestUtil.nextInt(r, 1, segCount);
+      w.optimize(limit);
+      assert w.getSegmentCount() <= limit: "limit=" + limit + " actual=" + w.getSegmentCount();
+    }
+  }
+
   public IndexReader getReader(boolean applyDeletions) throws IOException {
     getReaderCalled = true;
-    if (r.nextInt(4) == 2)
-      w.optimize();
+    if (r.nextInt(4) == 2) {
+      doRandomOptimize();
+    }
     // If we are writing with PreFlexRW, force a full
     // IndexReader.open so terms are sorted in codepoint
     // order during searching:
@@ -179,7 +193,7 @@ public class RandomIndexWriter implement
     // if someone isn't using getReader() API, we want to be sure to
     // maybeOptimize since presumably they might open a reader on the dir.
     if (getReaderCalled == false && r.nextInt(4) == 2) {
-      w.optimize();
+      doRandomOptimize();
     }
     w.close();
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Mon May  9 13:19:28 2011
@@ -142,12 +142,16 @@ public abstract class LuceneTestCase ext
   public static final String TEST_DIRECTORY = System.getProperty("tests.directory", "random");
   /** Get the number of times to run tests */
   public static final int TEST_ITER = Integer.parseInt(System.getProperty("tests.iter", "1"));
+  /** Get the minimum number of times to run tests until a failure happens */
+  public static final int TEST_ITER_MIN = Integer.parseInt(System.getProperty("tests.iter.min", Integer.toString(TEST_ITER)));
   /** Get the random seed for tests */
   public static final String TEST_SEED = System.getProperty("tests.seed", "random");
   /** whether or not nightly tests should run */
   public static final boolean TEST_NIGHTLY = Boolean.parseBoolean(System.getProperty("tests.nightly", "false"));
   /** the line file used by LineFileDocs */
   public static final String TEST_LINE_DOCS_FILE = System.getProperty("tests.linedocsfile", "europarl.lines.txt.gz");
+  /** whether or not to clean threads between test invocations: "false", "perMethod", "perClass" */
+  public static final String TEST_CLEAN_THREADS = System.getProperty("tests.cleanthreads", "perClass");
 
   private static final Pattern codecWithParam = Pattern.compile("(.*)\\(\\s*(\\d+)\\s*\\)");
 
@@ -342,10 +346,12 @@ public abstract class LuceneTestCase ext
   
   @AfterClass
   public static void afterClassLuceneTestCaseJ4() {
-    int rogueThreads = threadCleanup("test class");
-    if (rogueThreads > 0) {
-      // TODO: fail here once the leaks are fixed.
-      System.err.println("RESOURCE LEAK: test class left " + rogueThreads + " thread(s) running");
+    if (! "false".equals(TEST_CLEAN_THREADS)) {
+      int rogueThreads = threadCleanup("test class");
+      if (rogueThreads > 0) {
+        // TODO: fail here once the leaks are fixed.
+        System.err.println("RESOURCE LEAK: test class left " + rogueThreads + " thread(s) running");
+      }
     }
     String codecDescription;
     CodecProvider cp = CodecProvider.getDefault();
@@ -490,7 +496,7 @@ public abstract class LuceneTestCase ext
     assertTrue("ensure your setUp() calls super.setUp()!!!", setup);
     setup = false;
     BooleanQuery.setMaxClauseCount(savedBoolMaxClauseCount);
-    if (!getClass().getName().startsWith("org.apache.solr")) {
+    if ("perMethod".equals(TEST_CLEAN_THREADS)) {
       int rogueThreads = threadCleanup("test method: '" + getName() + "'");
       if (rogueThreads > 0) {
         System.err.println("RESOURCE LEAK: test method: '" + getName() 
@@ -534,7 +540,7 @@ public abstract class LuceneTestCase ext
     }
   }
 
-  private final static int THREAD_STOP_GRACE_MSEC = 1000;
+  private final static int THREAD_STOP_GRACE_MSEC = 50;
   // jvm-wide list of 'rogue threads' we found, so they only get reported once.
   private final static IdentityHashMap<Thread,Boolean> rogueThreads = new IdentityHashMap<Thread,Boolean>();
   
@@ -588,9 +594,6 @@ public abstract class LuceneTestCase ext
           Thread.setDefaultUncaughtExceptionHandler(null);
           if (!t.getName().startsWith("SyncThread")) // avoid zookeeper jre crash
             t.interrupt();
-          try {
-            t.join(THREAD_STOP_GRACE_MSEC);
-          } catch (InterruptedException e) { e.printStackTrace(); }
         }
       }
     }
@@ -1207,11 +1210,24 @@ public abstract class LuceneTestCase ext
       if (VERBOSE) {
         System.out.println("\nNOTE: running test " + arg0.getName());
       }
+      
+      // only print iteration info if the user requested more than one iterations
+      boolean verbose = VERBOSE && TEST_ITER > 1;
+      int lastIterFailed = -1;
       for (int i = 0; i < TEST_ITER; i++) {
-        if (VERBOSE && TEST_ITER > 1) {
+        if (verbose) {
           System.out.println("\nNOTE: running iter=" + (1+i) + " of " + TEST_ITER);
         }
         super.runChild(arg0, arg1);
+        if (testsFailed) {
+          lastIterFailed = i;
+          if (i == TEST_ITER_MIN - 1) {
+            if (verbose) {
+              System.out.println("\nNOTE: iteration " + lastIterFailed + " failed !");
+            }
+            break;
+          }
+        }
       }
     }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Mon May  9 13:19:28 2011
@@ -27,6 +27,7 @@ import java.io.OutputStream;
 import java.io.PrintStream;
 import java.lang.reflect.Method;
 import java.util.Enumeration;
+import java.util.List;
 import java.util.Random;
 import java.util.Map;
 import java.util.HashMap;
@@ -35,8 +36,11 @@ import java.util.zip.ZipFile;
 
 import org.junit.Assert;
 
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.CheckIndex;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
+import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.LogMergePolicy;
 import org.apache.lucene.index.MergeScheduler;
@@ -325,4 +329,13 @@ public class _TestUtil {
       throw new RuntimeException(e);
     }
   }
+  
+  /** Adds field info for a Document. */
+  public static void add(Document doc, FieldInfos fieldInfos) {
+    List<Fieldable> fields = doc.getFields();
+    for (Fieldable field : fields) {
+      fieldInfos.addOrUpdate(field.name(), field.isIndexed(), field.isTermVectorStored(), field.isStorePositionWithTermVector(),
+              field.isStoreOffsetWithTermVector(), field.getOmitNorms(), false, field.getOmitTermFreqAndPositions());
+    }
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java Mon May  9 13:19:28 2011
@@ -373,4 +373,28 @@ public class AutomatonTestUtil {
     a.removeDeadTransitions();
   }
 
+  /**
+   * Returns true if the language of this automaton is finite.
+   * <p>
+   * WARNING: this method is slow, it will blow up if the automaton is large.
+   * this is only used to test the correctness of our faster implementation.
+   */
+  public static boolean isFiniteSlow(Automaton a) {
+    if (a.isSingleton()) return true;
+    return isFiniteSlow(a.initial, new HashSet<State>());
+  }
+  
+  /**
+   * Checks whether there is a loop containing s. (This is sufficient since
+   * there are never transitions to dead states.)
+   */
+  // TODO: not great that this is recursive... in theory a
+  // large automata could exceed java's stack
+  private static boolean isFiniteSlow(State s, HashSet<State> path) {
+    path.add(s);
+    for (Transition t : s.getTransitions())
+      if (path.contains(t.to) || !isFiniteSlow(t.to, path)) return false;
+    path.remove(s);
+    return true;
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java Mon May  9 13:19:28 2011
@@ -35,13 +35,13 @@ public class TestNumericTokenStream exte
     final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
     final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
     final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
-    final BytesRef bytes = new BytesRef();
+    final BytesRef bytes = bytesAtt.getBytesRef();
     stream.reset();
     assertEquals(64, numericAtt.getValueSize());
     for (int shift=0; shift<64; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
       assertTrue("New token is available", stream.incrementToken());
       assertEquals("Shift value wrong", shift, numericAtt.getShift());
-      final int hash = bytesAtt.toBytesRef(bytes);
+      final int hash = bytesAtt.fillBytesRef();
       assertEquals("Hash incorrect", bytes.hashCode(), hash);
       assertEquals("Term is incorrectly encoded", lvalue & ~((1L << shift) - 1L), NumericUtils.prefixCodedToLong(bytes));
       assertEquals("Term raw value is incorrectly encoded", lvalue & ~((1L << shift) - 1L), numericAtt.getRawValue());
@@ -58,13 +58,13 @@ public class TestNumericTokenStream exte
     final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
     final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
     final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
-    final BytesRef bytes = new BytesRef();
+    final BytesRef bytes = bytesAtt.getBytesRef();
     stream.reset();
     assertEquals(32, numericAtt.getValueSize());
     for (int shift=0; shift<32; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
       assertTrue("New token is available", stream.incrementToken());
       assertEquals("Shift value wrong", shift, numericAtt.getShift());
-      final int hash = bytesAtt.toBytesRef(bytes);
+      final int hash = bytesAtt.fillBytesRef();
       assertEquals("Hash incorrect", bytes.hashCode(), hash);
       assertEquals("Term is incorrectly encoded", ivalue & ~((1 << shift) - 1), NumericUtils.prefixCodedToInt(bytes));
       assertEquals("Term raw value is incorrectly encoded", ((long) ivalue) & ~((1L << shift) - 1L), numericAtt.getRawValue());

Modified: lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Mon May  9 13:19:28 2011
@@ -74,12 +74,14 @@ public class Test2BTerms extends LuceneT
     }
 
     private final static class MyTermAttributeImpl extends AttributeImpl implements TermToBytesRefAttribute {
-      public int toBytesRef(BytesRef bs) {
-        bs.bytes = bytes.bytes;
-        bs.offset = bytes.offset;
-        bs.length = bytes.length;
+      public int fillBytesRef() {
         return bytes.hashCode();
       }
+      
+      public BytesRef getBytesRef() {
+        return bytes;
+      }
+
       @Override
       public void clear() {
       }

Modified: lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Mon May  9 13:19:28 2011
@@ -938,6 +938,40 @@ public class TestAddIndexes extends Luce
 
     assertTrue(c.failures.size() == 0);
   }
+ 
+  // LUCENE-2996: tests that addIndexes(IndexReader) applies existing deletes correctly.
+  public void testExistingDeletes() throws Exception {
+    Directory[] dirs = new Directory[2];
+    for (int i = 0; i < dirs.length; i++) {
+      dirs[i] = newDirectory();
+      IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer());
+      IndexWriter writer = new IndexWriter(dirs[i], conf);
+      Document doc = new Document();
+      doc.add(new Field("id", "myid", Store.NO, Index.NOT_ANALYZED_NO_NORMS));
+      writer.addDocument(doc);
+      writer.close();
+    }
+
+    IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer());
+    IndexWriter writer = new IndexWriter(dirs[0], conf);
+
+    // Now delete the document
+    writer.deleteDocuments(new Term("id", "myid"));
+    IndexReader r = IndexReader.open(dirs[1]);
+    try {
+      writer.addIndexes(r);
+    } finally {
+      r.close();
+    }
+    writer.commit();
+    assertEquals("Documents from the incoming index should not have been deleted", 1, writer.numDocs());
+    writer.close();
+
+    for (Directory dir : dirs) {
+      dir.close();
+    }
+
+  }
   
   private void addDocs3(IndexWriter writer, int numDocs) throws IOException {
     for (int i = 0; i < numDocs; i++) {
@@ -1045,8 +1079,8 @@ public class TestAddIndexes extends Luce
     IndexWriter w3 = new IndexWriter(dir, conf);
     w3.addIndexes(readers);
     w3.close();
-    
-    assertEquals("Only one compound segment should exist", 3, dir.listAll().length);
+    // we should now see segments_X, segments.gen,_Y.cfs, _Z.fnx
+    assertEquals("Only one compound segment should exist", 4, dir.listAll().length);
   }
   
 }