You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2011/12/04 02:14:58 UTC

svn commit: r1210041 [3/4] - in /lucene/dev/branches/solrcloud: ./ dev-tools/eclipse/ dev-tools/idea/lucene/contrib/ dev-tools/maven/ dev-tools/maven/solr/core/ lucene/ lucene/contrib/ lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appendi...

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsReader.java Sun Dec  4 01:14:52 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.index.codecs.l
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
@@ -263,8 +264,16 @@ public class Lucene40PostingsReader exte
     }
   }
 
+  static final int BUFFERSIZE = 64;
+
   // Decodes only docs
   private class SegmentDocsEnum extends DocsEnum {
+    final int[] docs = new int[BUFFERSIZE];
+    final int[] freqs = new int[BUFFERSIZE];
+    
+    int start = -1;
+    int count = 0;
+    
     final IndexInput freqIn;
     final IndexInput startFreqIn;
 
@@ -283,7 +292,7 @@ public class Lucene40PostingsReader exte
     int skipOffset;
 
     boolean skipped;
-    DefaultSkipListReader skipper;
+    Lucene40SkipListReader skipper;
 
     public SegmentDocsEnum(IndexInput freqIn) throws IOException {
       startFreqIn = freqIn;
@@ -294,7 +303,9 @@ public class Lucene40PostingsReader exte
       omitTF = fieldInfo.indexOptions == IndexOptions.DOCS_ONLY;
       if (omitTF) {
         freq = 1;
+        Arrays.fill(freqs, 1);
       }
+      
       storePayloads = fieldInfo.storePayloads;
       this.liveDocs = liveDocs;
       freqOffset = termState.freqOffset;
@@ -313,96 +324,133 @@ public class Lucene40PostingsReader exte
 
       skipped = false;
 
+      start = -1;
+      count = 0;
       return this;
     }
+    
+    @Override
+    public int freq() {
+      return freq;
+    }
 
     @Override
+    public int docID() {
+      return doc;
+    }
+    
+    @Override
     public int nextDoc() throws IOException {
-      //if (DEBUG) System.out.println("    stpr.nextDoc seg=" + segment + " fp=" + freqIn.getFilePointer());
-      while(true) {
-        if (ord == limit) {
-          //if (DEBUG) System.out.println("      return doc=" + NO_MORE_DOCS);
-          return doc = NO_MORE_DOCS;
+      while (++start < count) {
+        int d = docs[start];
+        if (liveDocs == null || liveDocs.get(d)) {
+          freq = freqs[start];
+          return doc = d;
         }
-
-        ord++;
-
-        // Decode next doc/freq pair
-        final int code = freqIn.readVInt();
-        // if (DEBUG) System.out.println("      code=" + code);
-        if (omitTF) {
-          accum += code;
+      }
+      return doc = refill();
+    }
+    
+    @Override
+    public int advance(int target) throws IOException {
+      // last doc in our buffer is >= target, binary search + next()
+      if (++start < count && docs[count-1] >= target) {
+        binarySearch(target);
+        return nextDoc();
+      }
+      
+      start = count; // buffer is consumed
+      
+      return doc = skipTo(target);
+    }
+    
+    private void binarySearch(int target) {
+      int hi = count - 1;
+      while (start <= hi) {
+        int mid = (hi + start) >>> 1;
+        int doc = docs[mid];
+        if (doc < target) {
+          start = mid + 1;
+        } else if (doc > target) {
+          hi = mid - 1;
         } else {
-          accum += code >>> 1;              // shift off low bit
-          if ((code & 1) != 0) {          // if low bit is set
-            freq = 1;                     // freq is one
-          } else {
-            freq = freqIn.readVInt();     // else read freq
-          }
-        }
-
-        if (liveDocs == null || liveDocs.get(accum)) {
+          start = mid;
           break;
         }
       }
-
-      //if (DEBUG) System.out.println("    stpr.nextDoc return doc=" + doc);
-      return (doc = accum);
+      start--;
     }
 
-    @Override
-    public int read() throws IOException {
-
-      final int[] docs = bulkResult.docs.ints;
-      final int[] freqs = bulkResult.freqs.ints;
-      int i = 0;
-      final int length = docs.length;
-      while (i < length && ord < limit) {
-        ord++;
-        // manually inlined call to next() for speed
-        final int code = freqIn.readVInt();
+    private int refill() throws IOException {
+      int doc = scanTo(0);
+      
+      int bufferSize = Math.min(docs.length, limit - ord);
+      start = -1;
+      count = bufferSize;
+      ord += bufferSize;
+      
+      if (omitTF)
+        fillDocs(bufferSize);
+      else
+        fillDocsAndFreqs(bufferSize);
+      
+      return doc;
+    }
+    
+    private int scanTo(int target) throws IOException {
+      while (ord++ < limit) {
+        int code = freqIn.readVInt();
         if (omitTF) {
           accum += code;
         } else {
-          accum += code >>> 1;              // shift off low bit
+          accum += code >>> 1;            // shift off low bit
           if ((code & 1) != 0) {          // if low bit is set
             freq = 1;                     // freq is one
           } else {
             freq = freqIn.readVInt();     // else read freq
           }
         }
-
-        if (liveDocs == null || liveDocs.get(accum)) {
-          docs[i] = doc = accum;
-          freqs[i] = freq;
-          ++i;
+        
+        if (accum >= target && (liveDocs == null || liveDocs.get(accum))) {
+          return accum;
         }
       }
       
-      return i;
+      return NO_MORE_DOCS;
     }
-
-    @Override
-    public int docID() {
-      return doc;
+    
+    private void fillDocs(int size) throws IOException {
+      int docs[] = this.docs;
+      for (int i = 0; i < size; i++) {
+        accum += freqIn.readVInt();
+        docs[i] = accum;
+      }
     }
-
-    @Override
-    public int freq() {
-      return freq;
+    
+    private void fillDocsAndFreqs(int size) throws IOException {
+      int docs[] = this.docs;
+      int freqs[] = this.freqs;
+      for (int i = 0; i < size; i++) {
+        int code = freqIn.readVInt();
+        accum += code >>> 1;                   // shift off low bit
+        docs[i] = accum;
+        if ((code & 1) != 0) {                 // if low bit is set
+          freqs[i] = 1;                        // freq is one
+        } else {
+          freqs[i] = freqIn.readVInt();        // else read freq
+        }
+      }
     }
 
-    @Override
-    public int advance(int target) throws IOException {
-
-      if ((target - skipInterval) >= doc && limit >= skipMinimum) {
+    private int skipTo(int target) throws IOException {
+      if ((target - skipInterval) >= accum && limit >= skipMinimum) {
 
         // There are enough docs in the posting to have
         // skip data, and it isn't too close.
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {
@@ -424,17 +472,11 @@ public class Lucene40PostingsReader exte
           // Skipper moved
 
           ord = newOrd;
-          doc = accum = skipper.getDoc();
+          accum = skipper.getDoc();
           freqIn.seek(skipper.getFreqPointer());
         }
       }
-        
-      // scan for the rest:
-      do {
-        nextDoc();
-      } while (target > doc);
-
-      return doc;
+      return scanTo(target);
     }
   }
 
@@ -460,7 +502,7 @@ public class Lucene40PostingsReader exte
     int posPendingCount;
 
     boolean skipped;
-    DefaultSkipListReader skipper;
+    Lucene40SkipListReader skipper;
     private long lazyProxPointer;
 
     public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
@@ -555,7 +597,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {
@@ -656,7 +698,7 @@ public class Lucene40PostingsReader exte
     boolean payloadPending;
 
     boolean skipped;
-    DefaultSkipListReader skipper;
+    Lucene40SkipListReader skipper;
     private BytesRef payload;
     private long lazyProxPointer;
 
@@ -754,7 +796,7 @@ public class Lucene40PostingsReader exte
 
         if (skipper == null) {
           // This is the first time this enum has ever been used for skipping -- do lazy init
-          skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+          skipper = new Lucene40SkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
         }
 
         if (!skipped) {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40PostingsWriter.java Sun Dec  4 01:14:52 2011
@@ -50,7 +50,7 @@ public final class Lucene40PostingsWrite
 
   final IndexOutput freqOut;
   final IndexOutput proxOut;
-  final DefaultSkipListWriter skipListWriter;
+  final Lucene40SkipListWriter skipListWriter;
   /** Expert: The fraction of TermDocs entries stored in skip tables,
    * used to accelerate {@link DocsEnum#advance(int)}.  Larger values result in
    * smaller indexes, greater acceleration, but fewer accelerable cases, while
@@ -113,7 +113,7 @@ public final class Lucene40PostingsWrite
 
     totalNumDocs = state.numDocs;
 
-    skipListWriter = new DefaultSkipListWriter(skipInterval,
+    skipListWriter = new Lucene40SkipListWriter(skipInterval,
                                                maxSkipLevels,
                                                state.numDocs,
                                                freqOut,

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReader.java Sun Dec  4 01:14:52 2011
@@ -411,34 +411,6 @@ public class SepPostingsReader extends P
     }
 
     @Override
-    public int read() throws IOException {
-      // TODO: -- switch to bulk read api in IntIndexInput
-      //System.out.println("sepdocs read");
-      final int[] docs = bulkResult.docs.ints;
-      final int[] freqs = bulkResult.freqs.ints;
-      int i = 0;
-      final int length = docs.length;
-      while (i < length && count < docFreq) {
-        count++;
-        // manually inlined call to next() for speed
-        //System.out.println("decode doc");
-        accum += docReader.next();
-        if (!omitTF) {
-          //System.out.println("decode freq");
-          freq = freqReader.next();
-        }
-
-        if (liveDocs == null || liveDocs.get(accum)) {
-          docs[i] = doc = accum;
-          freqs[i] = freq;
-          //System.out.println("  docs[" + i + "]=" + doc + " count=" + count + " dF=" + docFreq);
-          i++;
-        }
-      }
-      return i;
-    }
-
-    @Override
     public int freq() {
       return freq;
     }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Sun Dec  4 01:14:52 2011
@@ -18,13 +18,13 @@ package org.apache.lucene.index.codecs.s
  */
 
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.PostingsFormat;
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
 import org.apache.lucene.index.codecs.TermVectorsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40DocValuesFormat;
 
 /**
  * plain text index format.
@@ -39,7 +39,7 @@ public final class SimpleTextCodec exten
   private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
   private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
   // TODO: need a plain-text impl
-  private final DocValuesFormat docValues = new DefaultDocValuesFormat();
+  private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
   
   public SimpleTextCodec() {
     super("SimpleText");

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldInfosReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldInfosReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldInfosReader.java Sun Dec  4 01:14:52 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
 
 import static org.apache.lucene.index.codecs.simpletext.SimpleTextFieldInfosWriter.*;
 
@@ -57,45 +58,45 @@ public class SimpleTextFieldInfosReader 
     try {
       
       SimpleTextUtil.readLine(input, scratch);
-      assert scratch.startsWith(NUMFIELDS);
+      assert StringHelper.startsWith(scratch, NUMFIELDS);
       final int size = Integer.parseInt(readString(NUMFIELDS.length, scratch));
       FieldInfo infos[] = new FieldInfo[size];
 
       for (int i = 0; i < size; i++) {
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(NAME);
+        assert StringHelper.startsWith(scratch, NAME);
         String name = readString(NAME.length, scratch);
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(NUMBER);
+        assert StringHelper.startsWith(scratch, NUMBER);
         int fieldNumber = Integer.parseInt(readString(NUMBER.length, scratch));
 
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(ISINDEXED);
+        assert StringHelper.startsWith(scratch, ISINDEXED);
         boolean isIndexed = Boolean.parseBoolean(readString(ISINDEXED.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(STORETV);
+        assert StringHelper.startsWith(scratch, STORETV);
         boolean storeTermVector = Boolean.parseBoolean(readString(STORETV.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(STORETVPOS);
+        assert StringHelper.startsWith(scratch, STORETVPOS);
         boolean storePositionsWithTermVector = Boolean.parseBoolean(readString(STORETVPOS.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(STORETVOFF);
+        assert StringHelper.startsWith(scratch, STORETVOFF);
         boolean storeOffsetWithTermVector = Boolean.parseBoolean(readString(STORETVOFF.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(PAYLOADS);
+        assert StringHelper.startsWith(scratch, PAYLOADS);
         boolean storePayloads = Boolean.parseBoolean(readString(PAYLOADS.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(NORMS);
+        assert StringHelper.startsWith(scratch, NORMS);
         boolean omitNorms = !Boolean.parseBoolean(readString(NORMS.length, scratch));
 
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(DOCVALUES);
+        assert StringHelper.startsWith(scratch, DOCVALUES);
         String dvType = readString(DOCVALUES.length, scratch);
         final ValueType docValuesType;
         
@@ -106,7 +107,7 @@ public class SimpleTextFieldInfosReader 
         }
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(INDEXOPTIONS);
+        assert StringHelper.startsWith(scratch, INDEXOPTIONS);
         IndexOptions indexOptions = IndexOptions.valueOf(readString(INDEXOPTIONS.length, scratch));
 
         hasVectors |= storeTermVector;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Sun Dec  4 01:14:52 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
@@ -80,7 +81,7 @@ class SimpleTextFieldsReader extends Fie
           current = null;
           return null;
         }
-        if (scratch.startsWith(FIELD)) {
+        if (StringHelper.startsWith(scratch, FIELD)) {
           return current = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
         }
       }
@@ -270,7 +271,7 @@ class SimpleTextFieldsReader extends Fie
       while(true) {
         final long lineStart = in.getFilePointer();
         SimpleTextUtil.readLine(in, scratch);
-        if (scratch.startsWith(DOC)) {
+        if (StringHelper.startsWith(scratch, DOC)) {
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             in.seek(lineStart);
             if (!omitTF) {
@@ -282,15 +283,15 @@ class SimpleTextFieldsReader extends Fie
           docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
           termFreq = 0;
           first = false;
-        } else if (scratch.startsWith(FREQ)) {
+        } else if (StringHelper.startsWith(scratch, FREQ)) {
           UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+FREQ.length, scratch.length-FREQ.length, scratchUTF16);
           termFreq = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
-        } else if (scratch.startsWith(POS)) {
+        } else if (StringHelper.startsWith(scratch, POS)) {
           // skip termFreq++;
-        } else if (scratch.startsWith(PAYLOAD)) {
+        } else if (StringHelper.startsWith(scratch, PAYLOAD)) {
           // skip
         } else {
-          assert scratch.startsWith(TERM) || scratch.startsWith(FIELD) || scratch.startsWith(END): "scratch=" + scratch.utf8ToString();
+          assert StringHelper.startsWith(scratch, TERM) || StringHelper.startsWith(scratch, FIELD) || StringHelper.startsWith(scratch, END): "scratch=" + scratch.utf8ToString();
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             in.seek(lineStart);
             if (!omitTF) {
@@ -358,7 +359,7 @@ class SimpleTextFieldsReader extends Fie
       while(true) {
         final long lineStart = in.getFilePointer();
         SimpleTextUtil.readLine(in, scratch);
-        if (scratch.startsWith(DOC)) {
+        if (StringHelper.startsWith(scratch, DOC)) {
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             nextDocStart = lineStart;
             in.seek(posStart);
@@ -368,16 +369,16 @@ class SimpleTextFieldsReader extends Fie
           docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
           tf = 0;
           first = false;
-        } else if (scratch.startsWith(FREQ)) {
+        } else if (StringHelper.startsWith(scratch, FREQ)) {
           UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+FREQ.length, scratch.length-FREQ.length, scratchUTF16);
           tf = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
           posStart = in.getFilePointer();
-        } else if (scratch.startsWith(POS)) {
+        } else if (StringHelper.startsWith(scratch, POS)) {
           // skip
-        } else if (scratch.startsWith(PAYLOAD)) {
+        } else if (StringHelper.startsWith(scratch, PAYLOAD)) {
           // skip
         } else {
-          assert scratch.startsWith(TERM) || scratch.startsWith(FIELD) || scratch.startsWith(END);
+          assert StringHelper.startsWith(scratch, TERM) || StringHelper.startsWith(scratch, FIELD) || StringHelper.startsWith(scratch, END);
           if (!first && (liveDocs == null || liveDocs.get(docID))) {
             nextDocStart = lineStart;
             in.seek(posStart);
@@ -398,12 +399,12 @@ class SimpleTextFieldsReader extends Fie
     @Override
     public int nextPosition() throws IOException {
       SimpleTextUtil.readLine(in, scratch);
-      assert scratch.startsWith(POS): "got line=" + scratch.utf8ToString();
+      assert StringHelper.startsWith(scratch, POS): "got line=" + scratch.utf8ToString();
       UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+POS.length, scratch.length-POS.length, scratchUTF16_2);
       final int pos = ArrayUtil.parseInt(scratchUTF16_2.chars, 0, scratchUTF16_2.length);
       final long fp = in.getFilePointer();
       SimpleTextUtil.readLine(in, scratch);
-      if (scratch.startsWith(PAYLOAD)) {
+      if (StringHelper.startsWith(scratch, PAYLOAD)) {
         final int len = scratch.length - PAYLOAD.length;
         if (scratch2.bytes.length < len) {
           scratch2.grow(len);
@@ -477,7 +478,7 @@ class SimpleTextFieldsReader extends Fie
       OpenBitSet visitedDocs = new OpenBitSet();
       while(true) {
         SimpleTextUtil.readLine(in, scratch);
-        if (scratch.equals(END) || scratch.startsWith(FIELD)) {
+        if (scratch.equals(END) || StringHelper.startsWith(scratch, FIELD)) {
           if (lastDocsStart != -1) {
             b.add(lastTerm, new PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>(lastDocsStart,
                                                                                    new PairOutputs.Pair<Long,Long>((long) docFreq,
@@ -485,15 +486,15 @@ class SimpleTextFieldsReader extends Fie
             sumTotalTermFreq += totalTermFreq;
           }
           break;
-        } else if (scratch.startsWith(DOC)) {
+        } else if (StringHelper.startsWith(scratch, DOC)) {
           docFreq++;
           sumDocFreq++;
           UnicodeUtil.UTF8toUTF16(scratch.bytes, scratch.offset+DOC.length, scratch.length-DOC.length, scratchUTF16);
           int docID = ArrayUtil.parseInt(scratchUTF16.chars, 0, scratchUTF16.length);
           visitedDocs.set(docID);
-        } else if (scratch.startsWith(POS)) {
+        } else if (StringHelper.startsWith(scratch, POS)) {
           totalTermFreq++;
-        } else if (scratch.startsWith(TERM)) {
+        } else if (StringHelper.startsWith(scratch, TERM)) {
           if (lastDocsStart != -1) {
             b.add(lastTerm, new PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>(lastDocsStart,
                                                                                    new PairOutputs.Pair<Long,Long>((long) docFreq,

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosReader.java Sun Dec  4 01:14:52 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
 
 import static org.apache.lucene.index.codecs.simpletext.SimpleTextSegmentInfosWriter.*;
 
@@ -48,35 +49,31 @@ public class SimpleTextSegmentInfosReade
     final BytesRef scratch = new BytesRef();
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(VERSION);
+    assert StringHelper.startsWith(scratch, VERSION);
     infos.version = Long.parseLong(readString(VERSION.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(COUNTER);
+    assert StringHelper.startsWith(scratch, COUNTER);
     infos.counter = Integer.parseInt(readString(COUNTER.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(FNX_VERSION);
-    infos.setGlobalFieldMapVersion(Long.parseLong(readString(FNX_VERSION.length, scratch)));
-    
-    SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(NUM_USERDATA);
+    assert StringHelper.startsWith(scratch, NUM_USERDATA);
     int numUserData = Integer.parseInt(readString(NUM_USERDATA.length, scratch));
     infos.userData = new HashMap<String,String>();
 
     for (int i = 0; i < numUserData; i++) {
       SimpleTextUtil.readLine(input, scratch);
-      assert scratch.startsWith(USERDATA_KEY);
+      assert StringHelper.startsWith(scratch, USERDATA_KEY);
       String key = readString(USERDATA_KEY.length, scratch);
       
       SimpleTextUtil.readLine(input, scratch);
-      assert scratch.startsWith(USERDATA_VALUE);
+      assert StringHelper.startsWith(scratch, USERDATA_VALUE);
       String value = readString(USERDATA_VALUE.length, scratch);
       infos.userData.put(key, value);
     }
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(NUM_SEGMENTS);
+    assert StringHelper.startsWith(scratch, NUM_SEGMENTS);
     int numSegments = Integer.parseInt(readString(NUM_SEGMENTS.length, scratch));
     
     for (int i = 0; i < numSegments; i++) {
@@ -86,55 +83,55 @@ public class SimpleTextSegmentInfosReade
   
   public SegmentInfo readSegmentInfo(Directory directory, DataInput input, BytesRef scratch) throws IOException {
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_NAME);
+    assert StringHelper.startsWith(scratch, SI_NAME);
     final String name = readString(SI_NAME.length, scratch);
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_CODEC);
+    assert StringHelper.startsWith(scratch, SI_CODEC);
     final Codec codec = Codec.forName(readString(SI_CODEC.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_VERSION);
+    assert StringHelper.startsWith(scratch, SI_VERSION);
     final String version = readString(SI_VERSION.length, scratch);
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_DOCCOUNT);
+    assert StringHelper.startsWith(scratch, SI_DOCCOUNT);
     final int docCount = Integer.parseInt(readString(SI_DOCCOUNT.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_DELCOUNT);
+    assert StringHelper.startsWith(scratch, SI_DELCOUNT);
     final int delCount = Integer.parseInt(readString(SI_DELCOUNT.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_HASPROX);
+    assert StringHelper.startsWith(scratch, SI_HASPROX);
     final int hasProx = readTernary(SI_HASPROX.length, scratch);
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_HASVECTORS);
+    assert StringHelper.startsWith(scratch, SI_HASVECTORS);
     final int hasVectors = readTernary(SI_HASVECTORS.length, scratch);
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_USECOMPOUND);
+    assert StringHelper.startsWith(scratch, SI_USECOMPOUND);
     final boolean isCompoundFile = Boolean.parseBoolean(readString(SI_USECOMPOUND.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_DSOFFSET);
+    assert StringHelper.startsWith(scratch, SI_DSOFFSET);
     final int dsOffset = Integer.parseInt(readString(SI_DSOFFSET.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_DSSEGMENT);
+    assert StringHelper.startsWith(scratch, SI_DSSEGMENT);
     final String dsSegment = readString(SI_DSSEGMENT.length, scratch);
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_DSCOMPOUND);
+    assert StringHelper.startsWith(scratch, SI_DSCOMPOUND);
     final boolean dsCompoundFile = Boolean.parseBoolean(readString(SI_DSCOMPOUND.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_DELGEN);
+    assert StringHelper.startsWith(scratch, SI_DELGEN);
     final long delGen = Long.parseLong(readString(SI_DELGEN.length, scratch));
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_NUM_NORMGEN);
+    assert StringHelper.startsWith(scratch, SI_NUM_NORMGEN);
     final int numNormGen = Integer.parseInt(readString(SI_NUM_NORMGEN.length, scratch));
     final Map<Integer,Long> normGen;
     if (numNormGen == 0) {
@@ -143,28 +140,28 @@ public class SimpleTextSegmentInfosReade
       normGen = new HashMap<Integer,Long>();
       for (int i = 0; i < numNormGen; i++) {
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(SI_NORMGEN_KEY);
+        assert StringHelper.startsWith(scratch, SI_NORMGEN_KEY);
         int key = Integer.parseInt(readString(SI_NORMGEN_KEY.length, scratch));
         
         SimpleTextUtil.readLine(input, scratch);
-        assert scratch.startsWith(SI_NORMGEN_VALUE);
+        assert StringHelper.startsWith(scratch, SI_NORMGEN_VALUE);
         long value = Long.parseLong(readString(SI_NORMGEN_VALUE.length, scratch));
         normGen.put(key, value);
       }
     }
     
     SimpleTextUtil.readLine(input, scratch);
-    assert scratch.startsWith(SI_NUM_DIAG);
+    assert StringHelper.startsWith(scratch, SI_NUM_DIAG);
     int numDiag = Integer.parseInt(readString(SI_NUM_DIAG.length, scratch));
     Map<String,String> diagnostics = new HashMap<String,String>();
 
     for (int i = 0; i < numDiag; i++) {
       SimpleTextUtil.readLine(input, scratch);
-      assert scratch.startsWith(SI_DIAG_KEY);
+      assert StringHelper.startsWith(scratch, SI_DIAG_KEY);
       String key = readString(SI_DIAG_KEY.length, scratch);
       
       SimpleTextUtil.readLine(input, scratch);
-      assert scratch.startsWith(SI_DIAG_VALUE);
+      assert StringHelper.startsWith(scratch, SI_DIAG_VALUE);
       String value = readString(SI_DIAG_VALUE.length, scratch);
       diagnostics.put(key, value);
     }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextSegmentInfosWriter.java Sun Dec  4 01:14:52 2011
@@ -23,7 +23,6 @@ import java.util.Map.Entry;
 
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import org.apache.lucene.index.codecs.SegmentInfosWriter;
 import org.apache.lucene.store.ChecksumIndexOutput;
 import org.apache.lucene.store.Directory;
@@ -43,7 +42,6 @@ public class SimpleTextSegmentInfosWrite
 
   final static BytesRef VERSION             = new BytesRef("version ");
   final static BytesRef COUNTER             = new BytesRef("counter ");
-  final static BytesRef FNX_VERSION         = new BytesRef("global field map version ");
   final static BytesRef NUM_USERDATA        = new BytesRef("user data entries ");
   final static BytesRef USERDATA_KEY        = new BytesRef("  key ");
   final static BytesRef USERDATA_VALUE      = new BytesRef("  value ");
@@ -73,8 +71,8 @@ public class SimpleTextSegmentInfosWrite
     IndexOutput out = new ChecksumIndexOutput(dir.createOutput(segmentsFileName, new IOContext(new FlushInfo(infos.size(), infos.totalDocCount()))));
     boolean success = false;
     try {
-      // required preamble
-      out.writeInt(DefaultSegmentInfosWriter.FORMAT_CURRENT); // write FORMAT
+      // required preamble:
+      out.writeInt(SegmentInfos.FORMAT_CURRENT); // write FORMAT
       out.writeString(codecID); // write codecID
       // end preamble
       
@@ -87,11 +85,6 @@ public class SimpleTextSegmentInfosWrite
       SimpleTextUtil.write(out, COUNTER);
       SimpleTextUtil.write(out, Integer.toString(infos.counter), scratch);
       SimpleTextUtil.writeNewline(out);
-      
-      // global field map version
-      SimpleTextUtil.write(out, FNX_VERSION);
-      SimpleTextUtil.write(out, Long.toString(infos.getGlobalFieldMapVersion()), scratch);
-      SimpleTextUtil.writeNewline(out);
 
       // user data
       int numUserDataEntries = infos.getUserData() == null ? 0 : infos.getUserData().size();

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsReader.java Sun Dec  4 01:14:52 2011
@@ -36,6 +36,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 
 import static org.apache.lucene.index.codecs.simpletext.SimpleTextStoredFieldsWriter.*;
@@ -81,7 +82,7 @@ public class SimpleTextStoredFieldsReade
     offsets = new ArrayList<Long>();
     while (!scratch.equals(END)) {
       readLine();
-      if (scratch.startsWith(DOC)) {
+      if (StringHelper.startsWith(scratch, DOC)) {
         offsets.add(in.getFilePointer());
       }
     }
@@ -91,18 +92,18 @@ public class SimpleTextStoredFieldsReade
   public void visitDocument(int n, StoredFieldVisitor visitor) throws CorruptIndexException, IOException {
     in.seek(offsets.get(n));
     readLine();
-    assert scratch.startsWith(NUM);
+    assert StringHelper.startsWith(scratch, NUM);
     int numFields = parseIntAt(NUM.length);
     
     for (int i = 0; i < numFields; i++) {
       readLine();
-      assert scratch.startsWith(FIELD);
+      assert StringHelper.startsWith(scratch, FIELD);
       int fieldNumber = parseIntAt(FIELD.length);
       FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
       readLine();
-      assert scratch.startsWith(NAME);
+      assert StringHelper.startsWith(scratch, NAME);
       readLine();
-      assert scratch.startsWith(TYPE);
+      assert StringHelper.startsWith(scratch, TYPE);
       
       final BytesRef type;
       if (equalsAt(TYPE_STRING, scratch, TYPE.length)) {
@@ -127,7 +128,7 @@ public class SimpleTextStoredFieldsReade
           break;
         case NO:   
           readLine();
-          assert scratch.startsWith(VALUE);
+          assert StringHelper.startsWith(scratch, VALUE);
           break;
         case STOP: return;
       }
@@ -136,7 +137,7 @@ public class SimpleTextStoredFieldsReade
   
   private void readField(BytesRef type, FieldInfo fieldInfo, StoredFieldVisitor visitor) throws IOException {
     readLine();
-    assert scratch.startsWith(VALUE);
+    assert StringHelper.startsWith(scratch, VALUE);
     if (type == TYPE_STRING) {
       visitor.stringField(fieldInfo, new String(scratch.bytes, scratch.offset+VALUE.length, scratch.length-VALUE.length, "UTF-8"));
     } else if (type == TYPE_BINARY) {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java Sun Dec  4 01:14:52 2011
@@ -148,6 +148,8 @@ public class SimpleTextStoredFieldsWrite
         write(VALUE);
         write(bytes);
         newLine();
+      } else if (field.stringValue() == null) {
+        throw new IllegalArgumentException("field " + field.name() + " is stored but does not have binaryValue, stringValue nor numericValue");
       } else {
         write(TYPE_STRING);
         newLine();

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextTermVectorsReader.java Sun Dec  4 01:14:52 2011
@@ -46,6 +46,7 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 
 import static org.apache.lucene.index.codecs.simpletext.SimpleTextTermVectorsWriter.*;
@@ -88,7 +89,7 @@ public class SimpleTextTermVectorsReader
     offsets = new ArrayList<Long>();
     while (!scratch.equals(END)) {
       readLine();
-      if (scratch.startsWith(DOC)) {
+      if (StringHelper.startsWith(scratch, DOC)) {
         offsets.add(in.getFilePointer());
       }
     }
@@ -105,30 +106,30 @@ public class SimpleTextTermVectorsReader
     SortedMap<String,SimpleTVTerms> fields = new TreeMap<String,SimpleTVTerms>();
     in.seek(offsets.get(doc));
     readLine();
-    assert scratch.startsWith(NUMFIELDS);
+    assert StringHelper.startsWith(scratch, NUMFIELDS);
     int numFields = parseIntAt(NUMFIELDS.length);
     if (numFields == 0) {
       return null; // no vectors for this doc
     }
     for (int i = 0; i < numFields; i++) {
       readLine();
-      assert scratch.startsWith(FIELD);
+      assert StringHelper.startsWith(scratch, FIELD);
       int fieldNumber = parseIntAt(FIELD.length);
       
       readLine();
-      assert scratch.startsWith(FIELDNAME);
+      assert StringHelper.startsWith(scratch, FIELDNAME);
       String fieldName = readString(FIELDNAME.length, scratch);
       
       readLine();
-      assert scratch.startsWith(FIELDPOSITIONS);
+      assert StringHelper.startsWith(scratch, FIELDPOSITIONS);
       boolean positions = Boolean.parseBoolean(readString(FIELDPOSITIONS.length, scratch));
       
       readLine();
-      assert scratch.startsWith(FIELDOFFSETS);
+      assert StringHelper.startsWith(scratch, FIELDOFFSETS);
       boolean offsets = Boolean.parseBoolean(readString(FIELDOFFSETS.length, scratch));
       
       readLine();
-      assert scratch.startsWith(FIELDTERMCOUNT);
+      assert StringHelper.startsWith(scratch, FIELDTERMCOUNT);
       int termCount = parseIntAt(FIELDTERMCOUNT.length);
       
       SimpleTVTerms terms = new SimpleTVTerms();
@@ -136,7 +137,7 @@ public class SimpleTextTermVectorsReader
       
       for (int j = 0; j < termCount; j++) {
         readLine();
-        assert scratch.startsWith(TERMTEXT);
+        assert StringHelper.startsWith(scratch, TERMTEXT);
         BytesRef term = new BytesRef();
         int termLength = scratch.length - TERMTEXT.length;
         term.grow(termLength);
@@ -147,7 +148,7 @@ public class SimpleTextTermVectorsReader
         terms.terms.put(term, postings);
         
         readLine();
-        assert scratch.startsWith(TERMFREQ);
+        assert StringHelper.startsWith(scratch, TERMFREQ);
         postings.freq = parseIntAt(TERMFREQ.length);
         
         if (positions || offsets) {
@@ -163,17 +164,17 @@ public class SimpleTextTermVectorsReader
           for (int k = 0; k < postings.freq; k++) {
             if (positions) {
               readLine();
-              assert scratch.startsWith(POSITION);
+              assert StringHelper.startsWith(scratch, POSITION);
               postings.positions[k] = parseIntAt(POSITION.length);
             }
             
             if (offsets) {
               readLine();
-              assert scratch.startsWith(STARTOFFSET);
+              assert StringHelper.startsWith(scratch, STARTOFFSET);
               postings.startOffsets[k] = parseIntAt(STARTOFFSET.length);
               
               readLine();
-              assert scratch.startsWith(ENDOFFSET);
+              assert StringHelper.startsWith(scratch, ENDOFFSET);
               postings.endOffsets[k] = parseIntAt(ENDOFFSET.length);
             }
           }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Sun Dec  4 01:14:52 2011
@@ -35,6 +35,7 @@ import org.apache.lucene.util.AttributeS
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.BasicAutomata;
@@ -428,7 +429,7 @@ public final class FuzzyTermsEnum extend
      */
     @Override
     protected final AcceptStatus accept(BytesRef term) {
-      if (term.startsWith(prefixBytesRef)) {
+      if (StringHelper.startsWith(term, prefixBytesRef)) {
         UnicodeUtil.UTF8toUTF32(term, utf32);
         final float similarity = similarity(utf32.ints, realPrefixLength, utf32.length - realPrefixLength);
         if (similarity > minSimilarity) {

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Sun Dec  4 01:14:52 2011
@@ -101,24 +101,14 @@ public class MultiTermQueryWrapperFilter
     if (termsEnum.next() != null) {
       // fill into a FixedBitSet
       final FixedBitSet bitSet = new FixedBitSet(context.reader.maxDoc());
-      int termCount = 0;
       DocsEnum docsEnum = null;
       do {
-        termCount++;
         // System.out.println("  iter termCount=" + termCount + " term=" +
         // enumerator.term().toBytesString());
         docsEnum = termsEnum.docs(acceptDocs, docsEnum);
-        final DocsEnum.BulkReadResult result = docsEnum.getBulkResult();
-        while (true) {
-          final int count = docsEnum.read();
-          if (count != 0) {
-            final int[] docs = result.docs.ints;
-            for (int i = 0; i < count; i++) {
-              bitSet.set(docs[i]);
-            }
-          } else {
-            break;
-          }
+        int docid;
+        while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+          bitSet.set(docid);
         }
       } while (termsEnum.next() != null);
       // System.out.println("  done termCount=" + termCount);

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/PrefixTermsEnum.java Sun Dec  4 01:14:52 2011
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.StringHelper;
 
 /**
  * Subclass of FilteredTermEnum for enumerating all terms that match the
@@ -41,7 +42,7 @@ public class PrefixTermsEnum extends Fil
 
   @Override
   protected AcceptStatus accept(BytesRef term) {
-    if (term.startsWith(prefixRef)) {
+    if (StringHelper.startsWith(term, prefixRef)) {
       return AcceptStatus.YES;
     } else {
       return AcceptStatus.END;

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/SearcherManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/SearcherManager.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/SearcherManager.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/SearcherManager.java Sun Dec  4 01:14:52 2011
@@ -159,7 +159,13 @@ public final class SearcherManager {
       try {
         // IR.openIfChanged preserves NRT and applyDeletes
         // in the newly returned reader:
-        final IndexReader newReader = IndexReader.openIfChanged(currentSearcher.getIndexReader());
+        final IndexReader newReader;
+        final IndexSearcher searcherToReopen = acquire();
+        try {
+          newReader = IndexReader.openIfChanged(searcherToReopen.getIndexReader());
+        } finally {
+          release(searcherToReopen);
+        }
         if (newReader != null) {
           final IndexSearcher newSearcher = new IndexSearcher(newReader, es);
           boolean success = false;
@@ -246,6 +252,10 @@ public final class SearcherManager {
 
   private synchronized void swapSearcher(IndexSearcher newSearcher) throws IOException {
     ensureOpen();
+    // Don't allow un-closing!
+    if (currentSearcher == null && newSearcher != null) {
+      throw new AlreadyClosedException("this SearcherManager is closed");
+    }
     final IndexSearcher oldSearcher = currentSearcher;
     currentSearcher = newSearcher;
     release(oldSearcher);

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermScorer.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/search/TermScorer.java Sun Dec  4 01:14:52 2011
@@ -26,15 +26,6 @@ import org.apache.lucene.search.similari
  */
 final class TermScorer extends Scorer {
   private final DocsEnum docsEnum;
-  private int doc = -1;
-  private int freq;
-
-  private int pointer;
-  private int pointerMax;
-
-  private final int[] docs;
-  private final int[] freqs;
-  private final DocsEnum.BulkReadResult bulkResult;
   private final Similarity.ExactDocScorer docScorer;
   
   /**
@@ -52,76 +43,32 @@ final class TermScorer extends Scorer {
     super(weight);
     this.docScorer = docScorer;
     this.docsEnum = td;
-    bulkResult = td.getBulkResult();
-    docs = bulkResult.docs.ints;
-    freqs = bulkResult.freqs.ints;
-  }
-
-  @Override
-  public void score(Collector c) throws IOException {
-    score(c, Integer.MAX_VALUE, nextDoc());
-  }
-
-  // firstDocID is ignored since nextDoc() sets 'doc'
-  @Override
-  public boolean score(Collector c, int end, int firstDocID) throws IOException {
-    c.setScorer(this);
-    while (doc < end) {                           // for docs in window
-      //System.out.println("TS: collect doc=" + doc);
-      c.collect(doc);                      // collect score
-      if (++pointer >= pointerMax) {
-        pointerMax = docsEnum.read();  // refill
-        if (pointerMax != 0) {
-          pointer = 0;
-        } else {
-          doc = NO_MORE_DOCS;                // set to sentinel value
-          return false;
-        }
-      } 
-      doc = docs[pointer];
-      freq = freqs[pointer];
-    }
-    return true;
   }
 
   @Override
   public int docID() {
-    return doc;
+    return docsEnum.docID();
   }
 
   @Override
   public float freq() {
-    return freq;
+    return docsEnum.freq();
   }
 
   /**
    * Advances to the next document matching the query. <br>
-   * The iterator over the matching documents is buffered using
-   * {@link TermDocs#read(int[],int[])}.
    * 
    * @return the document matching the query or NO_MORE_DOCS if there are no more documents.
    */
   @Override
   public int nextDoc() throws IOException {
-    pointer++;
-    if (pointer >= pointerMax) {
-      pointerMax = docsEnum.read();  // refill
-      if (pointerMax != 0) {
-        pointer = 0;
-      } else {
-        return doc = NO_MORE_DOCS;
-      }
-    } 
-    doc = docs[pointer];
-    freq = freqs[pointer];
-    assert doc != NO_MORE_DOCS;
-    return doc;
+    return docsEnum.nextDoc();
   }
   
   @Override
   public float score() {
-    assert doc != NO_MORE_DOCS;
-    return docScorer.score(doc, freq);  
+    assert docID() != NO_MORE_DOCS;
+    return docScorer.score(docsEnum.docID(), docsEnum.freq());  
   }
 
   /**
@@ -135,24 +82,7 @@ final class TermScorer extends Scorer {
    */
   @Override
   public int advance(int target) throws IOException {
-    // first scan in cache
-    for (pointer++; pointer < pointerMax; pointer++) {
-      if (docs[pointer] >= target) {
-        freq = freqs[pointer];
-        return doc = docs[pointer];
-      }
-    }
-
-    // not found in readahead cache, seek underlying stream
-    int newDoc = docsEnum.advance(target);
-    //System.out.println("ts.advance docsEnum=" + docsEnum);
-    if (newDoc != NO_MORE_DOCS) {
-      doc = newDoc;
-      freq = docsEnum.freq();
-    } else {
-      doc = NO_MORE_DOCS;
-    }
-    return doc;
+    return docsEnum.advance(target);
   }
 
   /** Returns a string representation of this <code>TermScorer</code>. */

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/BytesRef.java Sun Dec  4 01:14:52 2011
@@ -118,30 +118,6 @@ public final class BytesRef implements C
     return new BytesRef(bytes, offset, length);
   }
 
-  private boolean sliceEquals(BytesRef other, int pos) {
-    if (pos < 0 || length - pos < other.length) {
-      return false;
-    }
-    int i = offset + pos;
-    int j = other.offset;
-    final int k = other.offset + other.length;
-    
-    while (j < k) {
-      if (bytes[i++] != other.bytes[j++]) {
-        return false;
-      }
-    }
-    
-    return true;
-  }
-  
-  public boolean startsWith(BytesRef other) {
-    return sliceEquals(other, 0);
-  }
-
-  public boolean endsWith(BytesRef other) {
-    return sliceEquals(other, length - other.length);
-  }
   
   /** Calculates the hash code as required by TermsHash during indexing.
    * <p>It is defined as:

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/InfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/InfoStream.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/InfoStream.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/InfoStream.java Sun Dec  4 01:14:52 2011
@@ -18,30 +18,47 @@ package org.apache.lucene.util;
  */
 
 import java.io.Closeable;
-import java.util.concurrent.atomic.AtomicInteger;
 
 /** @lucene.internal */
 public abstract class InfoStream implements Closeable {
-  // Used for printing messages
-  private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
-  protected final int messageID = MESSAGE_ID.getAndIncrement();
+
+  /** Instance of InfoStream that does no logging at all. */
+  public static final InfoStream NO_OUTPUT = new NoOutput();
+  private static final class NoOutput extends InfoStream {
+    @Override
+    public void message(String component, String message) {}
+    
+    @Override
+    public boolean isEnabled(String component) { return false; }
+
+    @Override
+    public void close() {}
+  }
   
   /** prints a message */
   public abstract void message(String component, String message);
   
-  private static InfoStream defaultInfoStream;
+  /** returns true if messages are enabled and should be posted to {@link #message}. */
+  public abstract boolean isEnabled(String component);
   
-  /** The default infoStream (possibly null) used
-   * by a newly instantiated classes.
+  private static InfoStream defaultInfoStream = NO_OUTPUT;
+  
+  /** The default {@code InfoStream} used by a newly instantiated classes.
    * @see #setDefault */
-  public static InfoStream getDefault() {
+  public static synchronized InfoStream getDefault() {
     return defaultInfoStream;
   }
   
-  /** Sets the default infoStream (possibly null) used
-   * by a newly instantiated classes.
-   * @see #setDefault */
-  public static void setDefault(InfoStream infoStream) {
+  /** Sets the default {@code InfoStream} used
+   * by a newly instantiated classes. It cannot be {@code null},
+   * to disable logging use {@link #NO_OUTPUT}.
+   * @see #getDefault */
+  public static synchronized void setDefault(InfoStream infoStream) {
+    if (infoStream == null) {
+      throw new IllegalArgumentException("Cannot set InfoStream default implementation to null. "+
+        "To disable logging use InfoStream.NO_OUTPUT");
+    }
     defaultInfoStream = infoStream;
   }
+  
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/PrintStreamInfoStream.java Sun Dec  4 01:14:52 2011
@@ -20,15 +20,25 @@ package org.apache.lucene.util;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Date;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * @lucene.internal
  */
 public class PrintStreamInfoStream extends InfoStream {
+  // Used for printing messages
+  private static final AtomicInteger MESSAGE_ID = new AtomicInteger();
+  protected final int messageID;
+  
   private final PrintStream stream;
   
   public PrintStreamInfoStream(PrintStream stream) {
+    this(stream, MESSAGE_ID.getAndIncrement());
+  }
+  
+  public PrintStreamInfoStream(PrintStream stream, int messageID) {
     this.stream = stream;
+    this.messageID = messageID;
   }
   
   @Override
@@ -37,6 +47,11 @@ public class PrintStreamInfoStream exten
   }
 
   @Override
+  public boolean isEnabled(String component) {
+    return true;
+  }
+
+  @Override
   public void close() throws IOException {
     if (!isSystemStream()) {
       stream.close();

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/StringHelper.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/StringHelper.java Sun Dec  4 01:14:52 2011
@@ -29,33 +29,21 @@ import java.util.StringTokenizer;
 public abstract class StringHelper {
 
   /**
-   * Compares two byte[] arrays, element by element, and returns the
+   * Compares two {@link BytesRef}, element by element, and returns the
    * number of elements common to both arrays.
    *
-   * @param bytes1 The first byte[] to compare
-   * @param bytes2 The second byte[] to compare
+   * @param left The first {@link BytesRef} to compare
+   * @param right The second {@link BytesRef} to compare
    * @return The number of common elements.
    */
-  public static int bytesDifference(byte[] bytes1, int len1, byte[] bytes2, int len2) {
-    int len = len1 < len2 ? len1 : len2;
+  public static int bytesDifference(BytesRef left, BytesRef right) {
+    int len = left.length < right.length ? left.length : right.length;
+    final byte[] bytesLeft = left.bytes;
+    final int offLeft = left.offset;
+    byte[] bytesRight = right.bytes;
+    final int offRight = right.offset;
     for (int i = 0; i < len; i++)
-      if (bytes1[i] != bytes2[i])
-        return i;
-    return len;
-  }
-  
-  /**
-   * Compares two byte[] arrays, element by element, and returns the
-   * number of elements common to both arrays.
-   *
-   * @param bytes1 The first byte[] to compare
-   * @param bytes2 The second byte[] to compare
-   * @return The number of common elements.
-   */
-  public static int bytesDifference(byte[] bytes1, int off1, int len1, byte[] bytes2, int off2, int len2) {
-    int len = len1 < len2 ? len1 : len2;
-    for (int i = 0; i < len; i++)
-      if (bytes1[i+off1] != bytes2[i+off2])
+      if (bytesLeft[i+offLeft] != bytesRight[i+offRight])
         return i;
     return len;
   }
@@ -108,4 +96,51 @@ public abstract class StringHelper {
       return s1.equals(s2);
     }
   }
+
+  /**
+   * Returns <code>true</code> iff the ref starts with the given prefix.
+   * Otherwise <code>false</code>.
+   * 
+   * @param ref
+   *          the {@link BytesRef} to test
+   * @param prefix
+   *          the expected prefix
+   * @return Returns <code>true</code> iff the ref starts with the given prefix.
+   *         Otherwise <code>false</code>.
+   */
+  public static boolean startsWith(BytesRef ref, BytesRef prefix) {
+    return sliceEquals(ref, prefix, 0);
+  }
+
+  /**
+   * Returns <code>true</code> iff the ref ends with the given suffix. Otherwise
+   * <code>false</code>.
+   * 
+   * @param ref
+   *          the {@link BytesRef} to test
+   * @param suffix
+   *          the expected suffix
+   * @return Returns <code>true</code> iff the ref ends with the given suffix.
+   *         Otherwise <code>false</code>.
+   */
+  public static boolean endsWith(BytesRef ref, BytesRef suffix) {
+    return sliceEquals(ref, suffix, ref.length - suffix.length);
+  }
+  
+  private static boolean sliceEquals(BytesRef sliceToTest, BytesRef other, int pos) {
+    if (pos < 0 || sliceToTest.length - pos < other.length) {
+      return false;
+    }
+    int i = sliceToTest.offset + pos;
+    int j = other.offset;
+    final int k = other.offset + other.length;
+    
+    while (j < k) {
+      if (sliceToTest.bytes[i++] != other.bytes[j++]) {
+        return false;
+      }
+    }
+    
+    return true;
+  }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/fst/FST.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/java/org/apache/lucene/util/fst/FST.java Sun Dec  4 01:14:52 2011
@@ -17,12 +17,22 @@ package org.apache.lucene.util.fst;
  * limitations under the License.
  */
 
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.fst.Builder.UnCompiledNode;
 
 // TODO: if FST is pure prefix trie we can do a more compact
@@ -334,6 +344,43 @@ public class FST<T> {
     out.writeVInt(bytes.length);
     out.writeBytes(bytes, 0, bytes.length);
   }
+  
+  /**
+   * Writes an automaton to a file. 
+   */
+  public void save(final File file) throws IOException {
+    boolean success = false;
+    OutputStream os = new BufferedOutputStream(new FileOutputStream(file));
+    try {
+      save(new OutputStreamDataOutput(os));
+      success = true;
+    } finally { 
+      if (success) { 
+        IOUtils.close(os);
+      } else {
+        IOUtils.closeWhileHandlingException(os); 
+      }
+    }
+  }
+
+  /**
+   * Reads an automaton from a file. 
+   */
+  public static <T> FST<T> read(File file, Outputs<T> outputs) throws IOException {
+    InputStream is = new BufferedInputStream(new FileInputStream(file));
+    boolean success = false;
+    try {
+      FST<T> fst = new FST<T>(new InputStreamDataInput(is), outputs);
+      success = true;
+      return fst;
+    } finally {
+      if (success) { 
+        IOUtils.close(is);
+      } else {
+        IOUtils.closeWhileHandlingException(is); 
+      }
+    }
+  }
 
   private void writeLabel(int v) throws IOException {
     assert v >= 0: "v=" + v;

Modified: lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/index/codecs/preflexrw/PreFlexFieldsWriter.java Sun Dec  4 01:14:52 2011
@@ -31,7 +31,7 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.index.codecs.lucene3x.Lucene3xPostingsFormat;
 import org.apache.lucene.index.codecs.lucene3x.TermInfo;
-import org.apache.lucene.index.codecs.lucene40.DefaultSkipListWriter;
+import org.apache.lucene.index.codecs.lucene40.Lucene40SkipListWriter;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -41,7 +41,7 @@ class PreFlexFieldsWriter extends Fields
   private final TermInfosWriter termsOut;
   private final IndexOutput freqOut;
   private final IndexOutput proxOut;
-  private final DefaultSkipListWriter skipListWriter;
+  private final Lucene40SkipListWriter skipListWriter;
   private final int totalNumDocs;
 
   public PreFlexFieldsWriter(SegmentWriteState state) throws IOException {
@@ -77,7 +77,7 @@ class PreFlexFieldsWriter extends Fields
       }
     }
 
-    skipListWriter = new DefaultSkipListWriter(termsOut.skipInterval,
+    skipListWriter = new Lucene40SkipListWriter(termsOut.skipInterval,
                                                termsOut.maxSkipLevels,
                                                totalNumDocs,
                                                freqOut,

Modified: lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java Sun Dec  4 01:14:52 2011
@@ -34,6 +34,8 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.ThrottledIndexOutput;
 import org.apache.lucene.util._TestUtil;
@@ -498,6 +500,12 @@ public class MockDirectoryWrapper extend
     return size;
   }
 
+  private boolean assertNoUnreferencedFilesOnClose = true;
+
+  public void setAssertNoUnrefencedFilesOnClose(boolean v) {
+    assertNoUnreferencedFilesOnClose = v;
+  }
+
   @Override
   public synchronized void close() throws IOException {
     maybeYield();
@@ -530,6 +538,29 @@ public class MockDirectoryWrapper extend
           System.out.println("\nNOTE: MockDirectoryWrapper: now run CheckIndex");
         } 
         _TestUtil.checkIndex(this);
+
+        if (assertNoUnreferencedFilesOnClose) {
+          // now look for unreferenced files:
+          String[] startFiles = listAll();
+          new IndexWriter(this, new IndexWriterConfig(LuceneTestCase.TEST_VERSION_CURRENT, null)).rollback();
+          String[] endFiles = listAll();
+
+          Arrays.sort(startFiles);
+          Arrays.sort(endFiles);
+
+          if (!Arrays.equals(startFiles, endFiles)) {
+            assert false : "unreferenced files: before delete:\n    " + Arrays.toString(startFiles) + "\n  after delete:\n    " + Arrays.toString(endFiles);
+          }
+
+          IndexReader ir1 = IndexReader.open(this);
+          int numDocs1 = ir1.numDocs();
+          ir1.close();
+          new IndexWriter(this, new IndexWriterConfig(LuceneTestCase.TEST_VERSION_CURRENT, null)).close();
+          IndexReader ir2 = IndexReader.open(this);
+          int numDocs2 = ir2.numDocs();
+          ir2.close();
+          assert numDocs1 == numDocs2 : "numDocs changed after opening/closing IW: before=" + numDocs1 + " after=" + numDocs2;
+        }
       }
     }
     delegate.close();

Modified: lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/FailOnNonBulkMergesInfoStream.java Sun Dec  4 01:14:52 2011
@@ -26,6 +26,11 @@ public class FailOnNonBulkMergesInfoStre
   @Override
   public void close() throws IOException {   
   }
+  
+  @Override
+  public boolean isEnabled(String component) {
+    return true;
+  }
 
   @Override
   public void message(String component, String message) {

Modified: lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/LuceneJUnitResultFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/LuceneJUnitResultFormatter.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/LuceneJUnitResultFormatter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/LuceneJUnitResultFormatter.java Sun Dec  4 01:14:52 2011
@@ -66,7 +66,8 @@ public class LuceneJUnitResultFormatter 
   private static final org.apache.lucene.store.Lock lock;
 
   static {
-    File lockDir = new File(System.getProperty("java.io.tmpdir"),
+    File lockDir = new File(
+        System.getProperty("tests.lockdir", System.getProperty("java.io.tmpdir")),
         "lucene_junit_lock");
     lockDir.mkdirs();
     if (!lockDir.exists()) {

Modified: lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test-framework/java/org/apache/lucene/util/NullInfoStream.java Sun Dec  4 01:14:52 2011
@@ -20,7 +20,7 @@ package org.apache.lucene.util;
 import java.io.IOException;
 
 /**
- * Prints nothing. Just to make sure tests pass w/ and without infostream
+ * Prints nothing. Just to make sure tests pass w/ and without enabled InfoStream
  * without actually making noise.
  * @lucene.experimental
  */
@@ -32,6 +32,12 @@ public class NullInfoStream extends Info
   }
   
   @Override
+  public boolean isEnabled(String component) {
+    assert component != null;
+    return true; // to actually enable logging, we just ignore on message()
+  }
+
+  @Override
   public void close() throws IOException {   
   }
 }

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Sun Dec  4 01:14:52 2011
@@ -177,7 +177,7 @@ public class Test2BTerms extends LuceneT
       Document doc = new Document();
       final MyTokenStream ts = new MyTokenStream(random, TERMS_PER_DOC);
 
-      FieldType customType = new FieldType(TextField.TYPE_STORED);
+      FieldType customType = new FieldType(TextField.TYPE_UNSTORED);
       customType.setIndexOptions(IndexOptions.DOCS_ONLY);
       customType.setOmitNorms(true);
       Field field = new Field("field", ts, customType);

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Sun Dec  4 01:14:52 2011
@@ -31,11 +31,6 @@ import org.apache.lucene.document.String
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
-import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
-import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
-import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
-import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
 import org.apache.lucene.index.codecs.DocValuesFormat;
 import org.apache.lucene.index.codecs.FieldInfosFormat;
 import org.apache.lucene.index.codecs.StoredFieldsFormat;
@@ -43,6 +38,11 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.SegmentInfosFormat;
 import org.apache.lucene.index.codecs.TermVectorsFormat;
 import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
+import org.apache.lucene.index.codecs.lucene40.Lucene40FieldInfosFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40DocValuesFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40SegmentInfosFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40StoredFieldsFormat;
+import org.apache.lucene.index.codecs.lucene40.Lucene40TermVectorsFormat;
 import org.apache.lucene.index.codecs.pulsing.Pulsing40PostingsFormat;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.PhraseQuery;
@@ -1082,7 +1082,7 @@ public class TestAddIndexes extends Luce
     w3.close();
     // we should now see segments_X,
     // segments.gen,_Y.cfs,_Y.cfe, _Z.fnx
-    assertEquals("Only one compound segment should exist, but got: " + Arrays.toString(dir.listAll()), 5, dir.listAll().length);
+    assertEquals("Only one compound segment should exist, but got: " + Arrays.toString(dir.listAll()), 4, dir.listAll().length);
     dir.close();
   }
   
@@ -1098,27 +1098,27 @@ public class TestAddIndexes extends Luce
 
     @Override
     public DocValuesFormat docValuesFormat() {
-      return new DefaultDocValuesFormat();
+      return new Lucene40DocValuesFormat();
     }
 
     @Override
     public StoredFieldsFormat storedFieldsFormat() {
-      return new DefaultStoredFieldsFormat();
+      return new Lucene40StoredFieldsFormat();
     }
     
     @Override
     public TermVectorsFormat termVectorsFormat() {
-      return new DefaultTermVectorsFormat();
+      return new Lucene40TermVectorsFormat();
     }
     
     @Override
     public FieldInfosFormat fieldInfosFormat() {
-      return new DefaultFieldInfosFormat();
+      return new Lucene40FieldInfosFormat();
     }
 
     @Override
     public SegmentInfosFormat segmentInfosFormat() {
-      return new DefaultSegmentInfosFormat();
+      return new Lucene40SegmentInfosFormat();
     }
   }
   

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Sun Dec  4 01:14:52 2011
@@ -583,8 +583,7 @@ public class TestBackwardsCompatibility 
                                "_0_1.del",
                                "_0_1.s" + contentFieldIndex,
                                "segments_2",
-                               "segments.gen",
-                               "_1.fnx"};
+                               "segments.gen"};
 
       String[] actual = dir.listAll();
       Arrays.sort(expected);

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestConsistentFieldNumbers.java Sun Dec  4 01:14:52 2011
@@ -179,7 +179,6 @@ public class TestConsistentFieldNumbers 
         FieldInfos fis1 = sis.info(0).getFieldInfos();
         assertEquals("f1", fis1.fieldInfo(0).name);
         assertEquals("f2", fis1.fieldInfo(1).name);
-        assertTrue(dir.fileExists("_1.fnx"));
       }
       
 
@@ -203,8 +202,6 @@ public class TestConsistentFieldNumbers 
         assertEquals("f1", fis2.fieldInfo(0).name);
         assertNull(fis2.fieldInfo(1));
         assertEquals("f3", fis2.fieldInfo(2).name);
-        assertFalse(dir.fileExists("_1.fnx"));
-        assertTrue(dir.fileExists("_2.fnx"));
       }
 
       {
@@ -232,9 +229,6 @@ public class TestConsistentFieldNumbers 
         assertEquals("f1", fis3.fieldInfo(0).name);
         assertEquals("f2", fis3.fieldInfo(1).name);
         assertEquals("f3", fis3.fieldInfo(2).name);
-        assertFalse(dir.fileExists("_1.fnx"));
-        assertTrue(dir.fileExists("_2.fnx"));
-        assertFalse(dir.fileExists("_3.fnx"));
       }
 
       {
@@ -262,9 +256,6 @@ public class TestConsistentFieldNumbers 
       assertEquals("f1", fis1.fieldInfo(0).name);
       assertEquals("f2", fis1.fieldInfo(1).name);
       assertEquals("f3", fis1.fieldInfo(2).name);
-      assertFalse(dir.fileExists("_1.fnx"));
-      assertTrue(dir.fileExists("_2.fnx"));
-      assertFalse(dir.fileExists("_3.fnx"));
       dir.close();
     }
   }

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestDocTermOrds.java Sun Dec  4 01:14:52 2011
@@ -37,6 +37,7 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util._TestUtil;
 
 // TODO:
@@ -262,7 +263,7 @@ public class TestDocTermOrds extends Luc
         final int[] docOrds = idToOrds[id];
         final List<Integer> newOrds = new ArrayList<Integer>();
         for(int ord : idToOrds[id]) {
-          if (termsArray[ord].startsWith(prefixRef)) {
+          if (StringHelper.startsWith(termsArray[ord], prefixRef)) {
             newOrds.add(ord);
           }
         }
@@ -332,7 +333,7 @@ public class TestDocTermOrds extends Luc
           TermsEnum termsEnum = terms.iterator(null);
           TermsEnum.SeekStatus result = termsEnum.seekCeil(prefixRef, false);
           if (result != TermsEnum.SeekStatus.END) {
-            assertFalse("term=" + termsEnum.term().utf8ToString() + " matches prefix=" + prefixRef.utf8ToString(), termsEnum.term().startsWith(prefixRef));
+            assertFalse("term=" + termsEnum.term().utf8ToString() + " matches prefix=" + prefixRef.utf8ToString(), StringHelper.startsWith(termsEnum.term(), prefixRef));
           } else {
             // ok
           }

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java Sun Dec  4 01:14:52 2011
@@ -43,7 +43,6 @@ import org.apache.lucene.document.String
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.DefaultTermVectorsReader;
 import org.apache.lucene.index.codecs.simpletext.SimpleTextCodec;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.FieldCache;
@@ -149,8 +148,6 @@ public class TestIndexWriter extends Luc
 
     public static void assertNoUnreferencedFiles(Directory dir, String message) throws IOException {
       String[] startFiles = dir.listAll();
-      SegmentInfos infos = new SegmentInfos();
-      infos.read(dir);
       new IndexWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random))).rollback();
       String[] endFiles = dir.listAll();
 

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java Sun Dec  4 01:14:52 2011
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.analysis.*;
 import org.apache.lucene.document.Document;
@@ -916,10 +917,8 @@ public class TestIndexWriterExceptions e
       } catch (RuntimeException re) {
         // Expected
       }
-      assertTrue(dir.fileExists("_1.fnx"));
       assertTrue(failure.failOnCommit && failure.failOnDeleteFile);
       w.rollback();
-      assertFalse(dir.fileExists("_1.fnx"));
       assertEquals(0, dir.listAll().length);
       dir.close();
     }
@@ -960,19 +959,23 @@ public class TestIndexWriterExceptions e
   // LUCENE-1429
   public void testOutOfMemoryErrorCausesCloseToFail() throws Exception {
 
-    final List<Throwable> thrown = new ArrayList<Throwable>();
+    final AtomicBoolean thrown = new AtomicBoolean(false);
     final Directory dir = newDirectory();
     final IndexWriter writer = new IndexWriter(dir,
         newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setInfoStream(new InfoStream() {
         @Override
         public void message(String component, final String message) {
-          if (message.startsWith("now flush at close") && 0 == thrown.size()) {
-            thrown.add(null);
+          if (message.startsWith("now flush at close") && thrown.compareAndSet(false, true)) {
             throw new OutOfMemoryError("fake OOME at " + message);
           }
         }
 
         @Override
+        public boolean isEnabled(String component) {
+          return true;
+        }
+        
+        @Override
         public void close() throws IOException {}
       }));
 

Modified: lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java?rev=1210041&r1=1210040&r2=1210041&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java (original)
+++ lucene/dev/branches/solrcloud/lucene/src/test/org/apache/lucene/index/TestTermVectorsReader.java Sun Dec  4 01:14:52 2011
@@ -32,7 +32,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.DefaultTermVectorsReader;
 import org.apache.lucene.index.codecs.TermVectorsReader;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;