You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2010/07/27 22:44:39 UTC

svn commit: r979860 [2/5] - in /lucene/dev/branches/realtime_search: ./ lucene/ lucene/contrib/ lucene/contrib/highlighter/src/test/ lucene/contrib/instantiated/src/test/org/apache/lucene/store/instantiated/ lucene/contrib/memory/src/test/org/apache/lu...

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Tue Jul 27 20:44:34 2010
@@ -40,12 +40,11 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
-import org.apache.lucene.util.ArrayUtil;
 
 /** Exposes flex API on a pre-flex index, as a codec. 
  * @lucene.experimental */
 public class PreFlexFields extends FieldsProducer {
-
+  
   private static final boolean DEBUG_SURROGATES = false;
 
   public TermInfosReader tis;
@@ -60,7 +59,7 @@ public class PreFlexFields extends Field
   private final int readBufferSize;
   private Directory cfsReader;
 
-  PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, int readBufferSize, int indexDivisor)
+  public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, int readBufferSize, int indexDivisor)
     throws IOException {
 
     si = info;
@@ -107,6 +106,15 @@ public class PreFlexFields extends Field
     this.dir = dir;
   }
 
+  // If this returns, we do the surrogates dance so that the
+  // terms are sorted by unicode sort order.  This should be
+  // true when segments are used for "normal" searching;
+  // it's only false during testing, to create a pre-flex
+  // index, using the test-only PreFlexRW.
+  protected boolean sortTermsByUnicode() {
+    return true;
+  }
+
   static void files(Directory dir, SegmentInfo info, Collection<String> files) throws IOException {
     files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.TERMS_EXTENSION));
     files.add(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.TERMS_INDEX_EXTENSION));
@@ -182,6 +190,12 @@ public class PreFlexFields extends Field
     if (cfsReader != null) {
       cfsReader.close();
     }
+    if (freqStream != null) {
+      freqStream.close();
+    }
+    if (proxStream != null) {
+      proxStream.close();
+    }
   }
 
   private class PreFlexFieldsEnum extends FieldsEnum {
@@ -228,7 +242,11 @@ public class PreFlexFields extends Field
     public Comparator<BytesRef> getComparator() {
       // Pre-flex indexes always sorted in UTF16 order, but
       // we remap on-the-fly to unicode order
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
+      if (sortTermsByUnicode()) {
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
+      } else {
+        return BytesRef.getUTF8SortedAsUTF16Comparator();
+      }
     }
   }
 
@@ -238,119 +256,225 @@ public class PreFlexFields extends Field
     private boolean skipNext;
     private BytesRef current;
 
-    private int[] surrogateSeekPending = new int[1];
-    private boolean[] surrogateDidSeekBack = new boolean[1];
-    private int surrogateSeekUpto;
-    private char[] pendingPrefix;
-
     private SegmentTermEnum seekTermEnum;
     private Term protoTerm;
+    
+    private static final byte UTF8_NON_BMP_LEAD = (byte) 0xf0;
+    private static final byte UTF8_HIGH_BMP_LEAD = (byte) 0xee;
+
+    // Returns true if the unicode char is "after" the
+    // surrogates in UTF16, ie >= U+E000 and <= U+FFFF:
+    private final boolean isHighBMPChar(byte[] b, int idx) {
+      return (b[idx] & UTF8_HIGH_BMP_LEAD) == UTF8_HIGH_BMP_LEAD;
+    }
+
+    // Returns true if the unicode char in the UTF8 byte
+    // sequence starting at idx encodes a char outside of
+    // BMP (ie what would be a surrogate pair in UTF16):
+    private final boolean isNonBMPChar(byte[] b, int idx) {
+      return (b[idx] & UTF8_NON_BMP_LEAD) == UTF8_NON_BMP_LEAD;
+    }
+
+    private final byte[] scratch = new byte[4];
+    private final BytesRef prevTerm = new BytesRef();
+    private final BytesRef scratchTerm = new BytesRef();
     private int newSuffixStart;
 
-    void reset(FieldInfo fieldInfo) throws IOException {
-      this.fieldInfo = fieldInfo;
-      protoTerm = new Term(fieldInfo.name);
-      if (termEnum == null) {
-        termEnum = getTermsDict().terms(protoTerm);
-        seekTermEnum = getTermsDict().terms(protoTerm);
+    // Swap in S, in place of E:
+    private boolean seekToNonBMP(SegmentTermEnum te, BytesRef term, int pos) throws IOException {
+      final int savLength = term.length;
+
+      assert term.offset == 0;
+
+      // The 3 bytes starting at downTo make up 1
+      // unicode character:
+      assert isHighBMPChar(term.bytes, pos);
+
+      // NOTE: we cannot make this assert, because
+      // AutomatonQuery legitimately sends us malformed UTF8
+      // (eg the UTF8 bytes with just 0xee)
+      // assert term.length >= pos + 3: "term.length=" + term.length + " pos+3=" + (pos+3) + " byte=" + Integer.toHexString(term.bytes[pos]) + " term=" + term.toString();
+
+      // Save the bytes && length, since we need to
+      // restore this if seek "back" finds no matching
+      // terms
+      if (term.bytes.length < 4+pos) {
+        term.grow(4+pos);
+      }
+
+      scratch[0] = term.bytes[pos];
+      scratch[1] = term.bytes[pos+1];
+      scratch[2] = term.bytes[pos+2];
+
+      term.bytes[pos] = (byte) 0xf0;
+      term.bytes[pos+1] = (byte) 0x90;
+      term.bytes[pos+2] = (byte) 0x80;
+      term.bytes[pos+3] = (byte) 0x80;
+      term.length = 4+pos;
+
+      if (DEBUG_SURROGATES) {
+        System.out.println("      try seek term=" + UnicodeUtil.toHexString(term.utf8ToString()));
+      }
+
+      // Seek "back":
+      getTermsDict().seekEnum(te, protoTerm.createTerm(term));
+
+      // Test if the term we seek'd to in fact found a
+      // surrogate pair at the same position as the E:
+      Term t2 = te.term();
+
+      // Cannot be null (or move to next field) because at
+      // "worst" it'd seek to the same term we are on now,
+      // unless we are being called from seek
+      if (t2 == null || t2.field() != fieldInfo.name) {
+        return false;
+      }
+
+      if (DEBUG_SURROGATES) {
+        System.out.println("      got term=" + UnicodeUtil.toHexString(t2.text()));
+      }
+
+      // Now test if prefix is identical and we found
+      // a non-BMP char at the same position:
+      BytesRef b2 = t2.bytes();
+      assert b2.offset == 0;
+
+      boolean matches;
+      if (b2.length >= term.length && isNonBMPChar(b2.bytes, pos)) {
+        matches = true;
+        for(int i=0;i<pos;i++) {
+          if (term.bytes[i] != b2.bytes[i]) {
+            matches = false;
+            break;
+          }
+        }              
       } else {
-        getTermsDict().seekEnum(termEnum, protoTerm);
+        matches = false;
       }
-      skipNext = true;
-      
-      surrogateSeekUpto = 0;
-      newSuffixStart = 0;
 
-      surrogatesDance();
+      // Restore term:
+      term.length = savLength;
+      term.bytes[pos] = scratch[0];
+      term.bytes[pos+1] = scratch[1];
+      term.bytes[pos+2] = scratch[2];
+
+      return matches;
     }
 
-    private void surrogatesDance() throws IOException {
-      
-      // Tricky: prior to 4.0, Lucene index sorted terms in
-      // UTF16 order, but as of 4.0 we sort by Unicode code
-      // point order.  These orders differ because of the
-      // surrrogates; so we have to fixup our enum, here, by
-      // carefully first seeking past the surrogates and
-      // then back again at the end.  The process is
-      // recursive, since any given term could have multiple
-      // new occurrences of surrogate pairs, so we use a
-      // stack to record the pending seek-backs.
+    // Seek type 2 "continue" (back to the start of the
+    // surrogates): scan the stripped suffix from the
+    // prior term, backwards. If there was an E in that
+    // part, then we try to seek back to S.  If that
+    // seek finds a matching term, we go there.
+    private boolean doContinue() throws IOException {
+
       if (DEBUG_SURROGATES) {
-        System.out.println("  dance start term=" + (termEnum.term() == null ? null : UnicodeUtil.toHexString(termEnum.term().text())));
+        System.out.println("  try cont");
       }
 
-      while(popPendingSeek());
-      while(pushNewSurrogate());
-    }
+      int downTo = prevTerm.length-1;
 
-    // only for debugging
-    private String getStack() {
-      if (surrogateSeekUpto == 0) {
-        return "null";
-      } else {
-        StringBuffer sb = new StringBuffer();
-        for(int i=0;i<surrogateSeekUpto;i++) {
-          if (i > 0) {
-            sb.append(' ');
+      boolean didSeek = false;
+      
+      final int limit = Math.min(newSuffixStart, scratchTerm.length-1);
+
+      while(downTo > limit) {
+
+        if (isHighBMPChar(prevTerm.bytes, downTo)) {
+
+          if (DEBUG_SURROGATES) {
+            System.out.println("    found E pos=" + downTo + " vs len=" + prevTerm.length);
+          }
+
+          if (seekToNonBMP(seekTermEnum, prevTerm, downTo)) {
+            // TODO: more efficient seek?
+            getTermsDict().seekEnum(termEnum, seekTermEnum.term());
+            //newSuffixStart = downTo+4;
+            newSuffixStart = downTo;
+            scratchTerm.copy(termEnum.term().bytes());
+            didSeek = true;
+            if (DEBUG_SURROGATES) {
+              System.out.println("      seek!");
+            }
+            break;
+          } else {
+            if (DEBUG_SURROGATES) {
+              System.out.println("      no seek");
+            }
           }
-          sb.append(surrogateSeekPending[i]);
         }
-        sb.append(" pendingSeekText=" + new String(pendingPrefix, 0, surrogateSeekPending[surrogateSeekUpto-1]));
-        return sb.toString();
+
+        // Shorten prevTerm in place so that we don't redo
+        // this loop if we come back here:
+        if ((prevTerm.bytes[downTo] & 0xc0) == 0xc0 || (prevTerm.bytes[downTo] & 0x80) == 0) {
+          prevTerm.length = downTo;
+        }
+        
+        downTo--;
       }
+
+      return didSeek;
     }
 
-    private boolean popPendingSeek() throws IOException {
+    // Look for seek type 3 ("pop"): if the delta from
+    // prev -> current was replacing an S with an E,
+    // we must now seek to beyond that E.  This seek
+    // "finishes" the dance at this character
+    // position.
+    private boolean doPop() throws IOException {
+
       if (DEBUG_SURROGATES) {
-        System.out.println("  check pop newSuffix=" + newSuffixStart + " stack=" + getStack());
+        System.out.println("  try pop");
       }
-      // if a .next() has advanced beyond the
-      // after-surrogates range we had last seeked to, we
-      // must seek back to the start and resume .next from
-      // there.  this pops the pending seek off the stack.
-      final Term t = termEnum.term();
-      if (surrogateSeekUpto > 0) {
-        final int seekPrefix = surrogateSeekPending[surrogateSeekUpto-1];
+
+      assert newSuffixStart <= prevTerm.length;
+      assert newSuffixStart < scratchTerm.length || newSuffixStart == 0;
+
+      if (prevTerm.length > newSuffixStart &&
+          isNonBMPChar(prevTerm.bytes, newSuffixStart) &&
+          isHighBMPChar(scratchTerm.bytes, newSuffixStart)) {
+
+        // Seek type 2 -- put 0xFF at this position:
+        scratchTerm.bytes[newSuffixStart] = (byte) 0xff;
+        scratchTerm.length = newSuffixStart+1;
+
         if (DEBUG_SURROGATES) {
-          System.out.println("    seekPrefix=" + seekPrefix);
+          System.out.println("    seek to term=" + UnicodeUtil.toHexString(scratchTerm.utf8ToString()) + " " + scratchTerm.toString());
         }
-        if (newSuffixStart < seekPrefix) {
-          assert pendingPrefix != null;
-          assert pendingPrefix.length > seekPrefix;
-          pendingPrefix[seekPrefix] = UnicodeUtil.UNI_SUR_HIGH_START;
-          pendingPrefix[1+seekPrefix] = UnicodeUtil.UNI_SUR_LOW_START;
-          Term t2 = protoTerm.createTerm(new BytesRef(pendingPrefix, 0, 2+seekPrefix));
+          
+        // TODO: more efficient seek?  can we simply swap
+        // the enums?
+        getTermsDict().seekEnum(termEnum, protoTerm.createTerm(scratchTerm));
+
+        final Term t2 = termEnum.term();
+
+        // We could hit EOF or different field since this
+        // was a seek "forward":
+        if (t2 != null && t2.field() == fieldInfo.name) {
+
           if (DEBUG_SURROGATES) {
-            System.out.println("    do pop; seek back to " + UnicodeUtil.toHexString(t2.text()));
+            System.out.println("      got term=" + UnicodeUtil.toHexString(t2.text()) + " " + t2.bytes());
           }
-          getTermsDict().seekEnum(termEnum, t2);
-          surrogateDidSeekBack[surrogateSeekUpto-1] = true;
 
-          // +2 because we don't want to re-check the
-          // surrogates we just seek'd back to
-          newSuffixStart = seekPrefix + 2;
+          final BytesRef b2 = t2.bytes();
+          assert b2.offset == 0;
+
+
+          // Set newSuffixStart -- we can't use
+          // termEnum's since the above seek may have
+          // done no scanning (eg, term was precisely
+          // and index term, or, was in the term seek
+          // cache):
+          scratchTerm.copy(b2);
+          setNewSuffixStart(prevTerm, scratchTerm);
+
           return true;
-        } else if (newSuffixStart == seekPrefix && surrogateDidSeekBack[surrogateSeekUpto-1] && t != null && t.field() == fieldInfo.name && t.text().charAt(seekPrefix) > UnicodeUtil.UNI_SUR_LOW_END) {
-          assert pendingPrefix != null;
-          assert pendingPrefix.length > seekPrefix;
-          pendingPrefix[seekPrefix] = 0xffff;
-          Term t2 = protoTerm.createTerm(new BytesRef(pendingPrefix, 0, 1+seekPrefix));
-          if (DEBUG_SURROGATES) {
-            System.out.println("    finish pop; seek fwd to " + UnicodeUtil.toHexString(t2.text()));
-          }
-          getTermsDict().seekEnum(termEnum, t2);
+        } else if (newSuffixStart != 0 || scratchTerm.length != 0) {
           if (DEBUG_SURROGATES) {
-            System.out.println("    found term=" + (termEnum.term() == null ? null : UnicodeUtil.toHexString(termEnum.term().text())));
-          }
-          surrogateSeekUpto--;
-
-          if (termEnum.term() == null || termEnum.term().field() != fieldInfo.name) {
-            // force pop
-            newSuffixStart = -1;
-          } else {
-            newSuffixStart = termEnum.newSuffixStart;
+            System.out.println("      got term=null (or next field)");
           }
-
+          newSuffixStart = 0;
+          scratchTerm.length = 0;
           return true;
         }
       }
@@ -358,117 +482,249 @@ public class PreFlexFields extends Field
       return false;
     }
 
-    private UnicodeUtil.UTF16Result termBuffer = new UnicodeUtil.UTF16Result();
-    private UnicodeUtil.UTF16Result seekBuffer = new UnicodeUtil.UTF16Result();
+    // Pre-flex indices store terms in UTF16 sort order, but
+    // certain queries require Unicode codepoint order; this
+    // method carefully seeks around surrogates to handle
+    // this impedance mismatch
+
+    private void surrogateDance() throws IOException {
+
+      if (!unicodeSortOrder) {
+        return;
+      }
+
+      // We are invoked after TIS.next() (by UTF16 order) to
+      // possibly seek to a different "next" (by unicode
+      // order) term.
+
+      // We scan only the "delta" from the last term to the
+      // current term, in UTF8 bytes.  We look at 1) the bytes
+      // stripped from the prior term, and then 2) the bytes
+      // appended to that prior term's prefix.
     
-    private boolean pushNewSurrogate() throws IOException {
+      // We don't care about specific UTF8 sequences, just
+      // the "category" of the UTF16 character.  Category S
+      // is a high/low surrogate pair (it non-BMP).
+      // Category E is any BMP char > UNI_SUR_LOW_END (and <
+      // U+FFFF). Category A is the rest (any unicode char
+      // <= UNI_SUR_HIGH_START).
+
+      // The core issue is that pre-flex indices sort the
+      // characters as ASE, while flex must sort as AES.  So
+      // when scanning, when we hit S, we must 1) seek
+      // forward to E and enum the terms there, then 2) seek
+      // back to S and enum all terms there, then 3) seek to
+      // after E.  Three different seek points (1, 2, 3).
+    
+      // We can easily detect S in UTF8: if a byte has
+      // prefix 11110 (0xf0), then that byte and the
+      // following 3 bytes encode a single unicode codepoint
+      // in S.  Similary,we can detect E: if a byte has
+      // prefix 1110111 (0xee), then that byte and the
+      // following 2 bytes encode a single unicode codepoint
+      // in E.
+
+      // Note that this is really a recursive process --
+      // maybe the char at pos 2 needs to dance, but any
+      // point in its dance, suddenly pos 4 needs to dance
+      // so you must finish pos 4 before returning to pos
+      // 2.  But then during pos 4's dance maybe pos 7 needs
+      // to dance, etc.  However, despite being recursive,
+      // we don't need to hold any state because the state
+      // can always be derived by looking at prior term &
+      // current term.
+
+      // TODO: can we avoid this copy?
+      if (termEnum.term() == null || termEnum.term().field() != fieldInfo.name) {
+        scratchTerm.length = 0;
+      } else {
+        scratchTerm.copy(termEnum.term().bytes());
+      }
+      
       if (DEBUG_SURROGATES) {
-        System.out.println("  check push newSuffix=" + newSuffixStart + " stack=" + getStack());
+        System.out.println("  dance");
+        System.out.println("    prev=" + UnicodeUtil.toHexString(prevTerm.utf8ToString()));
+        System.out.println("         " + prevTerm.toString());
+        System.out.println("    term=" + UnicodeUtil.toHexString(scratchTerm.utf8ToString()));
+        System.out.println("         " + scratchTerm.toString());
+      }
+
+      // This code assumes TermInfosReader/SegmentTermEnum
+      // always use BytesRef.offset == 0
+      assert prevTerm.offset == 0;
+      assert scratchTerm.offset == 0;
+
+      // Need to loop here because we may need to do multiple
+      // pops, and possibly a continue in the end, ie:
+      //
+      //  cont
+      //  pop, cont
+      //  pop, pop, cont
+      //  <nothing>
+      //
+
+      while(true) {
+        if (doContinue()) {
+          break;
+        } else {
+          if (!doPop()) {
+            break;
+          }
+        }
       }
-      final Term t = termEnum.term();
-      if (t == null || t.field() != fieldInfo.name) {
-        return false;
+
+      if (DEBUG_SURROGATES) {
+        System.out.println("  finish bmp ends");
       }
 
-      final BytesRef bytes = t.bytes();
-      UnicodeUtil.UTF8toUTF16(bytes.bytes, bytes.offset, bytes.length, termBuffer);
+      doPushes();
+    }
 
-      for(int i=Math.max(0,newSuffixStart);i<termBuffer.length;i++) {
-        final char ch = termBuffer.result[i];
-        if (ch >= UnicodeUtil.UNI_SUR_HIGH_START && ch <= UnicodeUtil.UNI_SUR_HIGH_END && (surrogateSeekUpto == 0 || i > surrogateSeekPending[surrogateSeekUpto-1])) {
 
-          if (DEBUG_SURROGATES) {
-            System.out.println("    found high surr 0x" + Integer.toHexString(ch) + " at pos=" + i);
-          }
+    // Look for seek type 1 ("push"): if the newly added
+    // suffix contains any S, we must try to seek to the
+    // corresponding E.  If we find a match, we go there;
+    // else we keep looking for additional S's in the new
+    // suffix.  This "starts" the dance, at this character
+    // position:
+    private void doPushes() throws IOException {
+
+      int upTo = newSuffixStart;
+      if (DEBUG_SURROGATES) {
+        System.out.println("  try push newSuffixStart=" + newSuffixStart + " scratchLen=" + scratchTerm.length);
+      }
 
-          // the next() that we just did read in a new
-          // suffix, containing a surrogate pair
+      while(upTo < scratchTerm.length) {
+        if (isNonBMPChar(scratchTerm.bytes, upTo) &&
+            (upTo > newSuffixStart ||
+             (upTo >= prevTerm.length ||
+              (!isNonBMPChar(prevTerm.bytes, upTo) &&
+               !isHighBMPChar(prevTerm.bytes, upTo))))) {
+
+          // A non-BMP char (4 bytes UTF8) starts here:
+          assert scratchTerm.length >= upTo + 4;
+          
+          final int savLength = scratchTerm.length;
+          scratch[0] = scratchTerm.bytes[upTo];
+          scratch[1] = scratchTerm.bytes[upTo+1];
+          scratch[2] = scratchTerm.bytes[upTo+2];
+
+          scratchTerm.bytes[upTo] = UTF8_HIGH_BMP_LEAD;
+          scratchTerm.bytes[upTo+1] = (byte) 0x80;
+          scratchTerm.bytes[upTo+2] = (byte) 0x80;
+          scratchTerm.length = upTo+3;
 
-          // seek forward to see if there are any terms with
-          // this same prefix, but with characters after the
-          // surrogate range; if so, we must first iterate
-          // them, then seek back to the surrogates
+          if (DEBUG_SURROGATES) {
+            System.out.println("    try seek 1 pos=" + upTo + " term=" + UnicodeUtil.toHexString(scratchTerm.utf8ToString()) + " " + scratchTerm.toString() + " len=" + scratchTerm.length);
+          }
 
-          char[] testPrefix = new char[i+2];
-          for(int j=0;j<i;j++) {
-            testPrefix[j] = termBuffer.result[j];
+          // Seek "forward":
+          // TODO: more efficient seek?
+          getTermsDict().seekEnum(seekTermEnum, protoTerm.createTerm(scratchTerm));
+
+          scratchTerm.bytes[upTo] = scratch[0];
+          scratchTerm.bytes[upTo+1] = scratch[1];
+          scratchTerm.bytes[upTo+2] = scratch[2];
+          scratchTerm.length = savLength;
+
+          // Did we find a match?
+          final Term t2 = seekTermEnum.term();
+            
+          if (DEBUG_SURROGATES) {
+            if (t2 == null) {
+              System.out.println("      hit term=null");
+            } else {
+              System.out.println("      hit term=" + UnicodeUtil.toHexString(t2.text()) + " " + (t2==null? null:t2.bytes()));
+            }
           }
-          testPrefix[i] = 1+UnicodeUtil.UNI_SUR_LOW_END;
 
-          getTermsDict().seekEnum(seekTermEnum, protoTerm.createTerm(new BytesRef(testPrefix, 0, i+1)));
+          // Since this was a seek "forward", we could hit
+          // EOF or a different field:
+          boolean matches;
 
-          Term t2 = seekTermEnum.term();
-          boolean isPrefix;
           if (t2 != null && t2.field() == fieldInfo.name) {
-
-            final BytesRef seekBytes = t2.bytes();
-            UnicodeUtil.UTF8toUTF16(seekBytes.bytes, seekBytes.offset, seekBytes.length, seekBuffer);
-
-            isPrefix = true;
-            if (DEBUG_SURROGATES) {
-              System.out.println("      seek found " + UnicodeUtil.toHexString(t2.text()));
-            }
-            for(int j=0;j<i;j++) {
-              if (testPrefix[j] != seekBuffer.result[j]) {
-                isPrefix = false;
-                break;
-              }
-            }
-            if (DEBUG_SURROGATES && !isPrefix) {
-              System.out.println("      no end terms");
+            final BytesRef b2 = t2.bytes();
+            assert b2.offset == 0;
+            if (b2.length >= upTo+3 && isHighBMPChar(b2.bytes, upTo)) {
+              matches = true;
+              for(int i=0;i<upTo;i++) {
+                if (scratchTerm.bytes[i] != b2.bytes[i]) {
+                  matches = false;
+                  break;
+                }
+              }              
+                
+            } else {
+              matches = false;
             }
           } else {
-            if (DEBUG_SURROGATES) {
-              System.out.println("      no end terms");
-            }
-            isPrefix = false;
+            matches = false;
           }
 
-          if (isPrefix) {
-            // we found a term, sharing the same prefix,
-            // with characters after the surrogates, so we
-            // must first enum those, and then return the
-            // the surrogates afterwards.  push that pending
-            // seek on the surrogates stack now:
-            pendingPrefix = testPrefix;
-
-            getTermsDict().seekEnum(termEnum, t2);
-
-            if (surrogateSeekUpto == surrogateSeekPending.length) {
-              surrogateSeekPending = ArrayUtil.grow(surrogateSeekPending);
-            }
-            if (surrogateSeekUpto == surrogateDidSeekBack.length) {
-              surrogateDidSeekBack = ArrayUtil.grow(surrogateDidSeekBack);
-            }
-            surrogateSeekPending[surrogateSeekUpto] = i;
-            surrogateDidSeekBack[surrogateSeekUpto] = false;
-            surrogateSeekUpto++;
+          if (matches) {
 
             if (DEBUG_SURROGATES) {
-              System.out.println("      do push " + i + "; end term=" + UnicodeUtil.toHexString(t2.text()));
+              System.out.println("      matches!");
             }
 
-            newSuffixStart = i+1;
-
-            return true;
+            // OK seek "back"
+            // TODO: more efficient seek?
+            getTermsDict().seekEnum(termEnum, seekTermEnum.term());
+
+            scratchTerm.copy(seekTermEnum.term().bytes());
+
+            // +3 because we don't need to check the char
+            // at upTo: we know it's > BMP
+            upTo += 3;
+
+            // NOTE: we keep iterating, now, since this
+            // can easily "recurse".  Ie, after seeking
+            // forward at a certain char position, we may
+            // find another surrogate in our [new] suffix
+            // and must then do another seek (recurse)
           } else {
-            // there are no terms after the surrogates, so
-            // we do nothing to the enum and just step
-            // through the surrogates like normal.  but we
-            // must keep iterating through the term, in case
-            // another surrogate pair appears later
+            upTo++;
           }
+        } else {
+          upTo++;
         }
       }
+    }
 
-      return false;
+    private boolean unicodeSortOrder;
+
+    void reset(FieldInfo fieldInfo) throws IOException {
+      //System.out.println("pff.reset te=" + termEnum);
+      this.fieldInfo = fieldInfo;
+      protoTerm = new Term(fieldInfo.name);
+      if (termEnum == null) {
+        termEnum = getTermsDict().terms(protoTerm);
+        seekTermEnum = getTermsDict().terms(protoTerm);
+        //System.out.println("  term=" + termEnum.term());
+      } else {
+        getTermsDict().seekEnum(termEnum, protoTerm);
+      }
+      skipNext = true;
+
+      unicodeSortOrder = sortTermsByUnicode();
+
+      final Term t = termEnum.term();
+      if (t != null && t.field() == fieldInfo.name) {
+        newSuffixStart = 0;
+        prevTerm.length = 0;
+        surrogateDance();
+      }
     }
 
     @Override
     public Comparator<BytesRef> getComparator() {
       // Pre-flex indexes always sorted in UTF16 order, but
       // we remap on-the-fly to unicode order
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
+      if (unicodeSortOrder) {
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
+      } else {
+        return BytesRef.getUTF8SortedAsUTF16Comparator();
+      }
     }
 
     @Override
@@ -484,7 +740,7 @@ public class PreFlexFields extends Field
     @Override
     public SeekStatus seek(BytesRef term, boolean useCache) throws IOException {
       if (DEBUG_SURROGATES) {
-        System.out.println("TE.seek() term=" + term.utf8ToString());
+        System.out.println("TE.seek target=" + UnicodeUtil.toHexString(term.utf8ToString()));
       }
       skipNext = false;
       final TermInfosReader tis = getTermsDict();
@@ -492,50 +748,142 @@ public class PreFlexFields extends Field
 
       assert termEnum != null;
 
-      if (termEnum == null) {
-        termEnum = tis.terms(t0);
-      } else {
-        tis.seekEnum(termEnum, t0);
-      }
-
-      surrogateSeekUpto = 0;
-      surrogatesDance();
+      tis.seekEnum(termEnum, t0);
 
       final Term t = termEnum.term();
 
-      final BytesRef tr = t == null ? null : t.bytes();
-
-      if (t != null && t.field() == fieldInfo.name && term.bytesEquals(tr)) {
-        current = tr;
+      if (t != null && t.field() == fieldInfo.name && term.bytesEquals(t.bytes())) {
+        // If we found an exact match, no need to do the
+        // surrogate dance
+        if (DEBUG_SURROGATES) {
+          System.out.println("  seek exact match");
+        }
+        current = t.bytes();
         return SeekStatus.FOUND;
       } else if (t == null || t.field() != fieldInfo.name) {
+
+        // TODO: maybe we can handle this like the next()
+        // into null?  set term as prevTerm then dance?
+
+        if (DEBUG_SURROGATES) {
+          System.out.println("  seek hit EOF");
+        }
+
+        // We hit EOF; try end-case surrogate dance: if we
+        // find an E, try swapping in S, backwards:
+        scratchTerm.copy(term);
+
+        assert scratchTerm.offset == 0;
+
+        for(int i=scratchTerm.length-1;i>=0;i--) {
+          if (isHighBMPChar(scratchTerm.bytes, i)) {
+            if (DEBUG_SURROGATES) {
+              System.out.println("    found E pos=" + i + "; try seek");
+            }
+
+            if (seekToNonBMP(seekTermEnum, scratchTerm, i)) {
+
+              scratchTerm.copy(seekTermEnum.term().bytes());
+              getTermsDict().seekEnum(termEnum, seekTermEnum.term());
+
+              newSuffixStart = 1+i;
+
+              doPushes();
+
+              // Found a match
+              // TODO: faster seek?
+              current = termEnum.term().bytes();
+              return SeekStatus.NOT_FOUND;
+            }
+          }
+        }
+        
+        if (DEBUG_SURROGATES) {
+          System.out.println("  seek END");
+        }
+
         current = null;
         return SeekStatus.END;
       } else {
-        current = tr;
-        return SeekStatus.NOT_FOUND;
+
+        // We found a non-exact but non-null term; this one
+        // is fun -- just treat it like next, by pretending
+        // requested term was prev:
+        prevTerm.copy(term);
+
+        if (DEBUG_SURROGATES) {
+          System.out.println("  seek hit non-exact term=" + UnicodeUtil.toHexString(t.text()));
+        }
+
+        final BytesRef br = t.bytes();
+        assert br.offset == 0;
+
+        setNewSuffixStart(term, br);
+
+        surrogateDance();
+
+        final Term t2 = termEnum.term();
+        if (t2 == null || t2.field() != fieldInfo.name) {
+          assert t2 == null || !t2.field().equals(fieldInfo.name); // make sure fields are in fact interned
+          current = null;
+          return SeekStatus.END;
+        } else {
+          current = t2.bytes();
+          assert !unicodeSortOrder || term.compareTo(current) < 0 : "term=" + UnicodeUtil.toHexString(term.utf8ToString()) + " vs current=" + UnicodeUtil.toHexString(current.utf8ToString());
+          return SeekStatus.NOT_FOUND;
+        }
+      }
+    }
+
+    private void setNewSuffixStart(BytesRef br1, BytesRef br2) {
+      final int limit = Math.min(br1.length, br2.length);
+      int lastStart = 0;
+      for(int i=0;i<limit;i++) {
+        if ((br1.bytes[br1.offset+i] & 0xc0) == 0xc0 || (br1.bytes[br1.offset+i] & 0x80) == 0) {
+          lastStart = i;
+        }
+        if (br1.bytes[br1.offset+i] != br2.bytes[br2.offset+i]) {
+          newSuffixStart = lastStart;
+          if (DEBUG_SURROGATES) {
+            System.out.println("    set newSuffixStart=" + newSuffixStart);
+          }
+          return;
+        }
+      }
+      newSuffixStart = limit;
+      if (DEBUG_SURROGATES) {
+        System.out.println("    set newSuffixStart=" + newSuffixStart);
       }
     }
 
     @Override
     public BytesRef next() throws IOException {
       if (DEBUG_SURROGATES) {
-        System.out.println("TE.next() skipNext=" + skipNext);
+        System.out.println("TE.next()");
       }
       if (skipNext) {
+        if (DEBUG_SURROGATES) {
+          System.out.println("  skipNext=true");
+        }
         skipNext = false;
         if (termEnum.term() == null) {
           return null;
+        } else if (termEnum.term().field() != fieldInfo.name) {
+          return null;
         } else {
           return current = termEnum.term().bytes();
         }
       }
+
+      // TODO: can we use STE's prevBuffer here?
+      prevTerm.copy(termEnum.term().bytes());
+
       if (termEnum.next() && termEnum.term().field() == fieldInfo.name) {
         newSuffixStart = termEnum.newSuffixStart;
         if (DEBUG_SURROGATES) {
-          System.out.println("  set newSuffixStart=" + newSuffixStart);
+          System.out.println("  newSuffixStart=" + newSuffixStart);
         }
-        surrogatesDance();
+        surrogateDance();
         final Term t = termEnum.term();
         if (t == null || t.field() != fieldInfo.name) {
           assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
@@ -545,12 +893,15 @@ public class PreFlexFields extends Field
         }
         return current;
       } else {
+        // This field is exhausted, but we have to give
+        // surrogateDance a chance to seek back:
         if (DEBUG_SURROGATES) {
-          System.out.println("  force pop");
+          System.out.println("  force cont");
         }
-        // force pop
-        newSuffixStart = -1;
-        surrogatesDance();
+        //newSuffixStart = prevTerm.length;
+        newSuffixStart = 0;
+        surrogateDance();
+        
         final Term t = termEnum.term();
         if (t == null || t.field() != fieldInfo.name) {
           assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
@@ -574,20 +925,32 @@ public class PreFlexFields extends Field
 
     @Override
     public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
-      if (reuse != null) {
-        return ((PreDocsEnum) reuse).reset(termEnum, skipDocs);        
+      PreDocsEnum docsEnum;
+      if (reuse == null || !(reuse instanceof PreDocsEnum)) {
+        docsEnum = new PreDocsEnum();
       } else {
-        return (new PreDocsEnum()).reset(termEnum, skipDocs);
+        docsEnum = (PreDocsEnum) reuse;
+        if (docsEnum.getFreqStream() != freqStream) {
+          docsEnum = new PreDocsEnum();
+        }
       }
+      return docsEnum.reset(termEnum, skipDocs);
     }
 
     @Override
     public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
-      if (reuse != null) {
-        return ((PreDocsAndPositionsEnum) reuse).reset(termEnum, skipDocs);        
+      PreDocsAndPositionsEnum docsPosEnum;
+      if (fieldInfo.omitTermFreqAndPositions) {
+        return null;
+      } else if (reuse == null || !(reuse instanceof PreDocsAndPositionsEnum)) {
+        docsPosEnum = new PreDocsAndPositionsEnum();
       } else {
-        return (new PreDocsAndPositionsEnum()).reset(termEnum, skipDocs);
+        docsPosEnum = (PreDocsAndPositionsEnum) reuse;
+        if (docsPosEnum.getFreqStream() != freqStream) {
+          docsPosEnum = new PreDocsAndPositionsEnum();
+        }
       }
+      return docsPosEnum.reset(termEnum, skipDocs);        
     }
   }
 
@@ -598,6 +961,10 @@ public class PreFlexFields extends Field
       docs = new SegmentTermDocs(freqStream, getTermsDict(), fieldInfos);
     }
 
+    IndexInput getFreqStream() {
+      return freqStream;
+    }
+
     public PreDocsEnum reset(SegmentTermEnum termEnum, Bits skipDocs) throws IOException {
       docs.setSkipDocs(skipDocs);
       docs.seek(termEnum);
@@ -650,6 +1017,10 @@ public class PreFlexFields extends Field
       pos = new SegmentTermPositions(freqStream, proxStream, getTermsDict(), fieldInfos);
     }
 
+    IndexInput getFreqStream() {
+      return freqStream;
+    }
+
     public DocsAndPositionsEnum reset(SegmentTermEnum termEnum, Bits skipDocs) throws IOException {
       pos.setSkipDocs(skipDocs);
       pos.seek(termEnum);

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java Tue Jul 27 20:44:34 2010
@@ -84,26 +84,16 @@ public final class SegmentTermEnum imple
       format = firstInt;
 
       // check that it is a format we can understand
-    if (format > FORMAT_MINIMUM)
-      throw new IndexFormatTooOldException(null, format, FORMAT_MINIMUM, FORMAT_CURRENT);
-    if (format < FORMAT_CURRENT)
-      throw new IndexFormatTooNewException(null, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      if (format > FORMAT_MINIMUM)
+        throw new IndexFormatTooOldException(null, format, FORMAT_MINIMUM, FORMAT_CURRENT);
+      if (format < FORMAT_CURRENT)
+        throw new IndexFormatTooNewException(null, format, FORMAT_MINIMUM, FORMAT_CURRENT);
 
       size = input.readLong();                    // read the size
       
-      if(format == -1){
-        if (!isIndex) {
-          indexInterval = input.readInt();
-          formatM1SkipInterval = input.readInt();
-        }
-        // switch off skipTo optimization for file format prior to 1.4rc2 in order to avoid a bug in 
-        // skipTo implementation of these versions
-        skipInterval = Integer.MAX_VALUE;
-      } else {
-        indexInterval = input.readInt();
-        skipInterval = input.readInt();
-        maxSkipLevels = input.readInt();
-      }
+      indexInterval = input.readInt();
+      skipInterval = input.readInt();
+      maxSkipLevels = input.readInt();
       assert indexInterval > 0: "indexInterval=" + indexInterval + " is negative; must be > 0";
       assert skipInterval > 0: "skipInterval=" + skipInterval + " is negative; must be > 0";
     }
@@ -132,18 +122,21 @@ public final class SegmentTermEnum imple
     position = p;
     termBuffer.set(t);
     prevBuffer.reset();
+    //System.out.println("  ste doSeek prev=" + prevBuffer.toTerm() + " this=" + this);
     termInfo.set(ti);
   }
 
   /** Increments the enumeration to the next element.  True if one exists.*/
   public final boolean next() throws IOException {
+    prevBuffer.set(termBuffer);
+    //System.out.println("  ste setPrev=" + prev() + " this=" + this);
+
     if (position++ >= size - 1) {
-      prevBuffer.set(termBuffer);
       termBuffer.reset();
+      //System.out.println("    EOF");
       return false;
     }
 
-    prevBuffer.set(termBuffer);
     termBuffer.read(input, fieldInfos);
     newSuffixStart = termBuffer.newSuffixStart;
 
@@ -168,6 +161,7 @@ public final class SegmentTermEnum imple
     if (isIndex)
       indexPointer += input.readVLong();	  // read index pointer
 
+    //System.out.println("  ste ret term=" + term());
     return true;
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java Tue Jul 27 20:44:34 2010
@@ -18,9 +18,10 @@ package org.apache.lucene.index.codecs.p
  */
 
 import java.io.IOException;
+import java.util.Comparator;
+
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.FieldInfos;
 
@@ -28,102 +29,65 @@ final class TermBuffer implements Clonea
 
   private String field;
   private Term term;                            // cached
-  private boolean dirty;                          // true if text was set externally (ie not read via UTF8 bytes)
 
-  private UnicodeUtil.UTF16Result text = new UnicodeUtil.UTF16Result();
   private BytesRef bytes = new BytesRef(10);
 
-  int newSuffixStart;
+  private static final Comparator<BytesRef> utf8AsUTF16Comparator = BytesRef.getUTF8SortedAsUTF16Comparator();
+
+  int newSuffixStart;                             // only valid right after .read is called
 
-  public final int compareTo(TermBuffer other) {
+  public int compareTo(TermBuffer other) {
     if (field == other.field) 	  // fields are interned
-      return compareChars(text.result, text.length, other.text.result, other.text.length);
+      return utf8AsUTF16Comparator.compare(bytes, other.bytes);
     else
       return field.compareTo(other.field);
   }
 
-  private static int compareChars(char[] chars1, int len1,
-                                  char[] chars2, int len2) {
-    final int end = len1 < len2 ? len1:len2;
-    for (int k = 0; k < end; k++) {
-      char c1 = chars1[k];
-      char c2 = chars2[k];
-      if (c1 != c2) {
-        return c1 - c2;
-      }
-    }
-    return len1 - len2;
-  }
-
-  public final void read(IndexInput input, FieldInfos fieldInfos)
+  public void read(IndexInput input, FieldInfos fieldInfos)
     throws IOException {
     this.term = null;                           // invalidate cache
-    int start = input.readVInt();
+    newSuffixStart = input.readVInt();
     int length = input.readVInt();
-    int totalLength = start + length;
+    int totalLength = newSuffixStart + length;
     if (bytes.bytes.length < totalLength) {
       bytes.grow(totalLength);
     }
-    if (dirty) {
-      // Fully convert all bytes since bytes is dirty
-      UnicodeUtil.UTF16toUTF8(text.result, 0, text.length, bytes);
-      bytes.length = totalLength;
-      input.readBytes(bytes.bytes, start, length);
-      UnicodeUtil.UTF8toUTF16(bytes.bytes, 0, totalLength, text);
-      dirty = false;
-    } else {
-      // Incrementally convert only the UTF8 bytes that are new:
-      bytes.length = totalLength;
-      input.readBytes(bytes.bytes, start, length);
-      UnicodeUtil.UTF8toUTF16(bytes.bytes, start, length, text);
-    }
-
-    while(true) {
-      newSuffixStart = text.offsets[start];
-      if (newSuffixStart != -1) {
-        break;
-      }
-      if (--start == 0) {
-        newSuffixStart = 0;
-        break;
-      }
-    }
+    bytes.length = totalLength;
+    input.readBytes(bytes.bytes, newSuffixStart, length);
     this.field = fieldInfos.fieldName(input.readVInt());
   }
 
-  public final void set(Term term) {
+  public void set(Term term) {
     if (term == null) {
       reset();
       return;
     }
-    
-    final BytesRef termBytes = term.bytes();
-    UnicodeUtil.UTF8toUTF16(termBytes.bytes, termBytes.offset, termBytes.length, text);
-    dirty = true;
+    bytes.copy(term.bytes());
     field = term.field();
     this.term = term;
   }
 
-  public final void set(TermBuffer other) {
-    text.copyText(other.text);
-    dirty = true;
+  public void set(TermBuffer other) {
     field = other.field;
-    term = other.term;
+    // dangerous to copy Term over, since the underlying
+    // BytesRef could subsequently be modified:
+    term = null;
+    bytes.copy(other.bytes);
   }
 
   public void reset() {
     field = null;
-    text.setLength(0);
     term = null;
-    dirty = true;
   }
 
   public Term toTerm() {
     if (field == null)                            // unset
       return null;
 
-    if (term == null)
-      term = new Term(field, new BytesRef(text.result, 0, text.length), false);
+    if (term == null) {
+      term = new Term(field, new BytesRef(bytes), false);
+      //term = new Term(field, bytes, false);
+    }
 
     return term;
   }
@@ -134,12 +98,7 @@ final class TermBuffer implements Clonea
     try {
       clone = (TermBuffer)super.clone();
     } catch (CloneNotSupportedException e) {}
-    clone.dirty = true;
-    clone.bytes = new BytesRef(10);
-    clone.text = new UnicodeUtil.UTF16Result();
-    clone.text.offsets = new int[text.offsets.length];
-    System.arraycopy(text.offsets, 0, clone.text.offsets, 0, text.offsets.length);
-    clone.text.copyText(text);
+    clone.bytes = new BytesRef(bytes);
     return clone;
   }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java Tue Jul 27 20:44:34 2010
@@ -23,30 +23,30 @@ package org.apache.lucene.index.codecs.p
  * indexing. */
 
 @Deprecated
-class TermInfo {
+public class TermInfo {
   /** The number of documents which contain the term. */
-  int docFreq = 0;
+  public int docFreq = 0;
 
-  long freqPointer = 0;
-  long proxPointer = 0;
-  int skipOffset;
+  public long freqPointer = 0;
+  public long proxPointer = 0;
+  public int skipOffset;
 
-  TermInfo() {}
+  public TermInfo() {}
 
-  TermInfo(int df, long fp, long pp) {
+  public TermInfo(int df, long fp, long pp) {
     docFreq = df;
     freqPointer = fp;
     proxPointer = pp;
   }
 
-  TermInfo(TermInfo ti) {
+  public TermInfo(TermInfo ti) {
     docFreq = ti.docFreq;
     freqPointer = ti.freqPointer;
     proxPointer = ti.proxPointer;
     skipOffset = ti.skipOffset;
   }
 
-  final void set(int docFreq,
+  public final void set(int docFreq,
                  long freqPointer, long proxPointer, int skipOffset) {
     this.docFreq = docFreq;
     this.freqPointer = freqPointer;
@@ -54,7 +54,7 @@ class TermInfo {
     this.skipOffset = skipOffset;
   }
 
-  final void set(TermInfo ti) {
+  public final void set(TermInfo ti) {
     docFreq = ti.docFreq;
     freqPointer = ti.freqPointer;
     proxPointer = ti.proxPointer;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Tue Jul 27 20:44:34 2010
@@ -119,9 +119,12 @@ public final class TermInfosReader {
           indexTerms = new Term[indexSize];
           indexInfos = new TermInfo[indexSize];
           indexPointers = new long[indexSize];
-        
-          for (int i = 0; indexEnum.next(); i++) {
+
+          for (int i=0;indexEnum.next(); i++) {
             indexTerms[i] = indexEnum.term();
+            assert indexTerms[i] != null;
+            assert indexTerms[i].text() != null;
+            assert indexTerms[i].field() != null;
             indexInfos[i] = indexEnum.termInfo();
             indexPointers[i] = indexEnum.indexPointer;
         
@@ -160,14 +163,14 @@ public final class TermInfosReader {
     return origEnum.maxSkipLevels;
   }
 
-  final void close() throws IOException {
+  void close() throws IOException {
     if (origEnum != null)
       origEnum.close();
     threadResources.close();
   }
 
   /** Returns the number of term/value pairs in the set. */
-  final long size() {
+  long size() {
     return size;
   }
 
@@ -183,12 +186,13 @@ public final class TermInfosReader {
 
 
   /** Returns the offset of the greatest index entry which is less than or equal to term.*/
-  private final int getIndexOffset(Term term) {
+  private int getIndexOffset(Term term) {
     int lo = 0;					  // binary search indexTerms[]
     int hi = indexTerms.length - 1;
 
     while (hi >= lo) {
       int mid = (lo + hi) >>> 1;
+      assert indexTerms[mid] != null : "indexTerms = " + indexTerms.length + " mid=" + mid;
       int delta = term.compareToUTF16(indexTerms[mid]);
       if (delta < 0)
 	hi = mid - 1;
@@ -200,7 +204,7 @@ public final class TermInfosReader {
     return hi;
   }
 
-  private final void seekEnum(SegmentTermEnum enumerator, int indexOffset) throws IOException {
+  private void seekEnum(SegmentTermEnum enumerator, int indexOffset) throws IOException {
     enumerator.seek(indexPointers[indexOffset],
                     ((long) indexOffset * totalIndexInterval) - 1,
                     indexTerms[indexOffset], indexInfos[indexOffset]);
@@ -231,6 +235,9 @@ public final class TermInfosReader {
   }
 
   TermInfo seekEnum(SegmentTermEnum enumerator, Term term, TermInfoAndOrd tiOrd) throws IOException {
+    if (size == 0) {
+      return null;
+    }
 
     // optimize sequential access: first try scanning cached enum w/o seeking
     if (enumerator.term() != null                 // term is at or past current
@@ -242,7 +249,6 @@ public final class TermInfosReader {
        // no need to seek
 
         final TermInfo ti;
-
         int numScans = enumerator.scanTo(term);
         if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
           ti = enumerator.termInfo();
@@ -279,6 +285,7 @@ public final class TermInfosReader {
     seekEnum(enumerator, indexPos);
     enumerator.scanTo(term);
     final TermInfo ti;
+
     if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
       ti = enumerator.termInfo();
       if (tiOrd == null) {
@@ -294,7 +301,7 @@ public final class TermInfosReader {
   }
 
   // called only from asserts
-  private final boolean sameTermInfo(TermInfo ti1, TermInfo ti2, SegmentTermEnum enumerator) {
+  private boolean sameTermInfo(TermInfo ti1, TermInfo ti2, SegmentTermEnum enumerator) {
     if (ti1.docFreq != ti2.docFreq) {
       return false;
     }
@@ -319,7 +326,7 @@ public final class TermInfosReader {
   }
 
   /** Returns the position of a Term in the set or -1. */
-  final long getPosition(Term term) throws IOException {
+  long getPosition(Term term) throws IOException {
     if (size == 0) return -1;
 
     ensureIndexIsRead();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java Tue Jul 27 20:44:34 2010
@@ -217,22 +217,40 @@ public class SepPostingsReaderImpl exten
   @Override
   public DocsEnum docs(FieldInfo fieldInfo, TermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
     final SepTermState termState = (SepTermState) _termState;
-    if (reuse == null) {
-      return (new SepDocsEnum()).init(fieldInfo, termState, skipDocs);
+    SepDocsEnum docsEnum;
+    if (reuse == null || !(reuse instanceof SepDocsEnum)) {
+      docsEnum = new SepDocsEnum();
     } else {
-      return ((SepDocsEnum) reuse).init(fieldInfo, termState, skipDocs);
+      docsEnum = (SepDocsEnum) reuse;
+      if (docsEnum.startDocIn != docIn) {
+        // If you are using ParellelReader, and pass in a
+        // reused DocsAndPositionsEnum, it could have come
+        // from another reader also using sep codec
+        docsEnum = new SepDocsEnum();        
+      }
     }
+
+    return docsEnum.init(fieldInfo, termState, skipDocs);
   }
 
   @Override
   public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, TermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
     assert !fieldInfo.omitTermFreqAndPositions;
     final SepTermState termState = (SepTermState) _termState;
-    if (reuse == null) {
-      return (new SepDocsAndPositionsEnum()).init(fieldInfo, termState, skipDocs);
+    SepDocsAndPositionsEnum postingsEnum;
+    if (reuse == null || !(reuse instanceof SepDocsAndPositionsEnum)) {
+      postingsEnum = new SepDocsAndPositionsEnum();
     } else {
-      return ((SepDocsAndPositionsEnum) reuse).init(fieldInfo, termState, skipDocs);
+      postingsEnum = (SepDocsAndPositionsEnum) reuse;
+      if (postingsEnum.startDocIn != docIn) {
+        // If you are using ParellelReader, and pass in a
+        // reused DocsAndPositionsEnum, it could have come
+        // from another reader also using sep codec
+        postingsEnum = new SepDocsAndPositionsEnum();        
+      }
     }
+
+    return postingsEnum.init(fieldInfo, termState, skipDocs);
   }
 
   class SepDocsEnum extends DocsEnum {
@@ -253,6 +271,7 @@ public class SepPostingsReaderImpl exten
     private final IntIndexInput.Index docIndex;
     private final IntIndexInput.Index freqIndex;
     private final IntIndexInput.Index posIndex;
+    private final IntIndexInput startDocIn;
 
     // TODO: -- should we do hasProx with 2 different enum classes?
 
@@ -260,6 +279,7 @@ public class SepPostingsReaderImpl exten
     SepSkipListReader skipper;
 
     SepDocsEnum() throws IOException {
+      startDocIn = docIn;
       docReader = docIn.reader();
       docIndex = docIn.index();
       if (freqIn != null) {
@@ -439,6 +459,8 @@ public class SepPostingsReaderImpl exten
     private final IntIndexInput.Index docIndex;
     private final IntIndexInput.Index freqIndex;
     private final IntIndexInput.Index posIndex;
+    private final IntIndexInput startDocIn;
+
     private long payloadOffset;
 
     private int pendingPosCount;
@@ -452,6 +474,7 @@ public class SepPostingsReaderImpl exten
     private boolean posSeekPending;
 
     SepDocsAndPositionsEnum() throws IOException {
+      startDocIn = docIn;
       docReader = docIn.reader();
       docIndex = docIn.index();
       freqReader = freqIn.reader();

Propchange: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Tue Jul 27 20:44:34 2010
@@ -1,5 +1,6 @@
-/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:943137,949730,957490,960490,961612
-/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:953476-978809
+/lucene/dev/branches/branch_3x/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:943137,949730,957490,960490,961612,979161
+/lucene/dev/branches/preflexfixes/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:967125-979432
+/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:953476-979858
 /lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:824912-931101
 /lucene/java/branches/lucene_2_9/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:909334,948516
 /lucene/java/trunk/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java:924483-924731,924781,925176-925462

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSDirectory.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSDirectory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/FSDirectory.java Tue Jul 27 20:44:34 2010
@@ -33,6 +33,8 @@ import java.util.Collections;
 import static java.util.Collections.synchronizedSet;
 import java.util.HashSet;
 import java.util.Set;
+
+import org.apache.lucene.store.SimpleFSDirectory.SimpleFSIndexInput;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.Constants;
@@ -126,6 +128,12 @@ public abstract class FSDirectory extend
   protected final Set<String> staleFiles = synchronizedSet(new HashSet<String>()); // Files written, but not yet sync'ed
   private int chunkSize = DEFAULT_READ_CHUNK_SIZE; // LUCENE-1566
 
+  /**
+   * Chunk size used to read when using FileChannel API. If an attempt to read a
+   * large file is made without limiting the chunk size, an OOM may occur.
+   */
+  private static final long CHANNEL_CHUNK_SIZE = 1 << 21; // Use 2MB chunk size - LUCENE-2537
+
   // returns the canonical version of the directory, creating it if it doesn't exist.
   private static File getCanonicalPath(File file) throws IOException {
     return new File(file.getCanonicalPath());
@@ -441,7 +449,7 @@ public abstract class FSDirectory extend
       try {
         input = new FileInputStream(new File(directory, src)).getChannel();
         output = new FileOutputStream(new File(target.directory, dest)).getChannel();
-        output.transferFrom(input, 0, input.size());
+        copy(input, output, input.size());
       } catch (IOException ioe) {
         priorException = ioe;
       } finally {
@@ -451,6 +459,25 @@ public abstract class FSDirectory extend
       super.copy(to, src, dest);
     }
   }
+
+  /**
+   * Copies the content of a given {@link FileChannel} to a destination one. The
+   * copy is done in chunks of 2MB because if transferFrom is used without a
+   * limit when copying a very large file, then an OOM may be thrown (depends on
+   * the state of the RAM in the machine, as well as the OS used). Performance
+   * measurements showed that chunk sizes larger than 2MB do not result in much
+   * faster file copy, therefore we limit the size to be safe with different
+   * file sizes and systems.
+   */
+  static void copy(FileChannel input, FileChannel output, long numBytes) throws IOException {
+    long pos = output.position();
+    long writeTo = numBytes + pos;
+    while (pos < writeTo) {
+      pos += output.transferFrom(input, pos, Math.min(CHANNEL_CHUNK_SIZE, writeTo - pos));
+    }
+    // transferFrom does not change the position of the channel. Need to change it manually
+    output.position(pos);
+  }
   
   protected static class FSIndexOutput extends BufferedIndexOutput {
     private final FSDirectory parent;
@@ -472,6 +499,28 @@ public abstract class FSDirectory extend
     }
 
     @Override
+    public void copyBytes(DataInput input, long numBytes) throws IOException {
+      // Optimized copy only if the number of bytes to copy is larger than the
+      // buffer size, and the given IndexInput supports FileChannel copying ..
+      // NOTE: the below check relies on NIOIndexInput extending Simple. If that
+      // changes in the future, we should change the check as well.
+      if (numBytes > BUFFER_SIZE && input instanceof SimpleFSIndexInput) {
+        // flush any bytes in the buffer
+        flush();
+        // do the optimized copy
+        FileChannel in = ((SimpleFSIndexInput) input).file.getChannel();
+        FileChannel out = file.getChannel();
+        copy(in, out, numBytes);
+        // corrects the position in super (BufferedIndexOutput), so that calls
+        // to getFilePointer will return the correct pointer.
+        // Perhaps a specific method is better?
+        super.seek(out.position());
+      } else {
+        super.copyBytes(input, numBytes);
+      }
+    }
+    
+    @Override
     public void close() throws IOException {
       // only close the file if it has not been closed yet
       if (isOpen) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java Tue Jul 27 20:44:34 2010
@@ -125,7 +125,7 @@ public class SimpleFSDirectory extends F
           final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
               "OutOfMemoryError likely caused by the Sun VM Bug described in "
               + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
-              + "with a a value smaller than the current chunks size (" + chunkSize + ")");
+              + "with a value smaller than the current chunks size (" + chunkSize + ")");
           outOfMemoryError.initCause(e);
           throw outOfMemoryError;
         }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRef.java Tue Jul 27 20:44:34 2010
@@ -331,12 +331,17 @@ public final class BytesRef implements C
           // We know the terms are not equal, but, we may
           // have to carefully fixup the bytes at the
           // difference to match UTF16's sort order:
+          
+          // NOTE: instead of moving supplementary code points (0xee and 0xef) to the unused 0xfe and 0xff, 
+          // we move them to the unused 0xfc and 0xfd [reserved for future 6-byte character sequences]
+          // this reserves 0xff for preflex's term reordering (surrogate dance), and if unicode grows such
+          // that 6-byte sequences are needed we have much bigger problems anyway.
           if (aByte >= 0xee && bByte >= 0xee) {
             if ((aByte & 0xfe) == 0xee) {
-              aByte += 0x10;
+              aByte += 0xe;
             }
             if ((bByte&0xfe) == 0xee) {
-              bByte += 0x10;
+              bByte += 0xe;
             }
           }
           return aByte - bByte;
@@ -346,10 +351,6 @@ public final class BytesRef implements C
       // One is a prefix of the other, or, they are equal:
       return a.length - b.length;
     }
-
-    public boolean equals(Object other) {
-      return this == other;
-    }
   }
 
   public void writeExternal(ObjectOutput out)

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/analysis/TestCharTokenizers.java Tue Jul 27 20:44:34 2010
@@ -39,7 +39,8 @@ public class TestCharTokenizers extends 
     Random newRandom = newRandom();
     // create random input
     int num = 1024 + newRandom.nextInt(1024);
-    for (int i = 1; i < num*_TestUtil.getRandomMultiplier(); i++) {
+    num *= RANDOM_MULTIPLIER;
+    for (int i = 1; i < num; i++) {
       builder.append("\ud801\udc1cabc");
       if((i % 10) == 0)
         builder.append(" ");

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestBinaryDocument.java Tue Jul 27 20:44:34 2010
@@ -2,9 +2,7 @@ package org.apache.lucene.document;
 
 import org.apache.lucene.util.LuceneTestCase;
 
-import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.store.MockRAMDirectory;
 
@@ -58,8 +56,7 @@ public class TestBinaryDocument extends 
     
     /** add the doc to a ram index */
     MockRAMDirectory dir = new MockRAMDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir, 
-        new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
+    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir);
     writer.addDocument(doc);
     
     /** open a reader and fetch the document */ 
@@ -98,8 +95,7 @@ public class TestBinaryDocument extends 
     
     /** add the doc to a ram index */
     MockRAMDirectory dir = new MockRAMDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir, 
-        new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
+    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir);
     writer.addDocument(doc);
     
     /** open a reader and fetch the document */ 

Propchange: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDateTools.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Tue Jul 27 20:44:34 2010
@@ -1,5 +1,6 @@
-/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/document/TestDateTools.java:943137,949730,957490,960490,961612
-/lucene/dev/trunk/lucene/src/test/org/apache/lucene/document/TestDateTools.java:953476-978809
+/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/document/TestDateTools.java:943137,949730,957490,960490,961612,979161
+/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/document/TestDateTools.java:967125-979432
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/document/TestDateTools.java:953476-979858
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestDateTools.java:824912-931101
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/document/TestDateTools.java:748824
 /lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestDateTools.java:829134,829881,831036,896850,909334,948516

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestDocument.java Tue Jul 27 20:44:34 2010
@@ -1,8 +1,6 @@
 package org.apache.lucene.document;
 
-import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
@@ -155,8 +153,7 @@ public class TestDocument extends Lucene
    */
   public void testGetValuesForIndexedDocument() throws Exception {
     RAMDirectory dir = new RAMDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir,
-        new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
+    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir);
     writer.addDocument(makeDocumentWithFields());
     IndexReader reader = writer.getReader();
     
@@ -234,8 +231,7 @@ public class TestDocument extends Lucene
         Field.Index.NOT_ANALYZED));
     
     RAMDirectory dir = new RAMDirectory();
-    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir,
-        new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()));
+    RandomIndexWriter writer = new RandomIndexWriter(newRandom(), dir);
     writer.addDocument(doc);
     field.setValue("id2");
     writer.addDocument(doc);

Propchange: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/document/TestNumberTools.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Tue Jul 27 20:44:34 2010
@@ -1,5 +1,6 @@
-/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/document/TestNumberTools.java:943137,949730,957490,960490,961612
-/lucene/dev/trunk/lucene/src/test/org/apache/lucene/document/TestNumberTools.java:953476-978809
+/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/document/TestNumberTools.java:943137,949730,957490,960490,961612,979161
+/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/document/TestNumberTools.java:967125-979432
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/document/TestNumberTools.java:953476-979858
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/document/TestNumberTools.java:824912-931101
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/document/TestNumberTools.java:748824
 /lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/document/TestNumberTools.java:829134,829881,831036,896850,909334,948516

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/RandomIndexWriter.java Tue Jul 27 20:44:34 2010
@@ -17,20 +17,18 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.util.Random;
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Random;
 
-import org.apache.lucene.util._TestUtil;
-import org.apache.lucene.store.Directory;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.intblock.IntBlockCodec;
-import org.apache.lucene.index.codecs.preflex.PreFlexCodec;
-import org.apache.lucene.index.codecs.pulsing.PulsingCodec;
-import org.apache.lucene.index.codecs.sep.SepCodec;
-import org.apache.lucene.index.codecs.standard.StandardCodec;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCaseJ4;
+import org.apache.lucene.util.Version;
+import org.apache.lucene.util._TestUtil;
 
 /** Silly class that randomizes the indexing experience.  EG
  *  it may swap in a different merge policy/scheduler; may
@@ -45,32 +43,48 @@ public class RandomIndexWriter implement
   int docCount;
   int flushAt;
 
-  public RandomIndexWriter(Random r, Directory dir, IndexWriterConfig c) throws IOException {
-    this.r = r;
-    if (r.nextBoolean()) {
-      c.setMergePolicy(new LogDocMergePolicy());
-    }
-    if (r.nextBoolean()) {
-      c.setMergeScheduler(new SerialMergeScheduler());
+  // Randomly calls Thread.yield so we mixup thread scheduling
+  private static final class MockIndexWriter extends IndexWriter {
+
+    private final Random r;
+
+    public MockIndexWriter(Random r,Directory dir, IndexWriterConfig conf) throws IOException {
+      super(dir, conf);
+      this.r = r;
     }
-    if (r.nextBoolean()) {
-      c.setMaxBufferedDocs(_TestUtil.nextInt(r, 2, 1000));
+
+    @Override
+    boolean testPoint(String name) {
+      if (r.nextInt(4) == 2)
+        Thread.yield();
+      return true;
     }
-    if (r.nextBoolean()) {
-      c.setTermIndexInterval(_TestUtil.nextInt(r, 1, 1000));
-    }
-    
-    if (c.getMergePolicy() instanceof LogMergePolicy) {
-      LogMergePolicy logmp = (LogMergePolicy) c.getMergePolicy();
-      logmp.setUseCompoundDocStore(r.nextBoolean());
-      logmp.setUseCompoundFile(r.nextBoolean());
-      logmp.setCalibrateSizeByDeletes(r.nextBoolean());
-    }
-    
-    c.setReaderPooling(r.nextBoolean());
-    c.setCodecProvider(new RandomCodecProvider(r));
-    w = new IndexWriter(dir, c);
+  }
+
+  /** create a RandomIndexWriter with a random config: Uses TEST_VERSION_CURRENT and MockAnalyzer */
+  public RandomIndexWriter(Random r, Directory dir) throws IOException {
+    this(r, dir, LuceneTestCaseJ4.newIndexWriterConfig(r, LuceneTestCaseJ4.TEST_VERSION_CURRENT, new MockAnalyzer()));
+  }
+  
+  /** create a RandomIndexWriter with a random config: Uses TEST_VERSION_CURRENT */
+  public RandomIndexWriter(Random r, Directory dir, Analyzer a) throws IOException {
+    this(r, dir, LuceneTestCaseJ4.newIndexWriterConfig(r, LuceneTestCaseJ4.TEST_VERSION_CURRENT, a));
+  }
+  
+  /** create a RandomIndexWriter with a random config */
+  public RandomIndexWriter(Random r, Directory dir, Version v, Analyzer a) throws IOException {
+    this(r, dir, LuceneTestCaseJ4.newIndexWriterConfig(r, v, a));
+  }
+  
+  /** create a RandomIndexWriter with the provided config */
+  public RandomIndexWriter(Random r, Directory dir, IndexWriterConfig c) throws IOException {
+    this.r = r;
+    w = new MockIndexWriter(r, dir, c);
     flushAt = _TestUtil.nextInt(r, 10, 1000);
+    if (LuceneTestCaseJ4.VERBOSE) {
+      System.out.println("RIW config=" + w.getConfig());
+      System.out.println("codec default=" + CodecProvider.getDefaultCodec());
+    }
   } 
 
   public void addDocument(Document doc) throws IOException {
@@ -89,14 +103,27 @@ public class RandomIndexWriter implement
     w.deleteDocuments(term);
   }
   
+  public void commit() throws CorruptIndexException, IOException {
+    w.commit();
+  }
+  
   public int maxDoc() {
     return w.maxDoc();
   }
 
   public IndexReader getReader() throws IOException {
-    if (r.nextBoolean()) {
+    // If we are writing with PreFlexRW, force a full
+    // IndexReader.open so terms are sorted in codepoint
+    // order during searching:
+    if (!w.codecs.getWriter(null).name.equals("PreFlex") && r.nextBoolean()) {
+      if (LuceneTestCaseJ4.VERBOSE) {
+        System.out.println("RIW.getReader: use NRT reader");
+      }
       return w.getReader();
     } else {
+      if (LuceneTestCaseJ4.VERBOSE) {
+        System.out.println("RIW.getReader: open new reader");
+      }
       w.commit();
       return IndexReader.open(w.getDirectory(), new KeepOnlyLastCommitDeletionPolicy(), r.nextBoolean(), _TestUtil.nextInt(r, 1, 10));
     }
@@ -112,22 +139,4 @@ public class RandomIndexWriter implement
   public void optimize() throws IOException {
     w.optimize();
   }
-  
-  class RandomCodecProvider extends CodecProvider {
-    final String codec;
-    
-    RandomCodecProvider(Random random) {
-      register(new StandardCodec());
-      register(new IntBlockCodec());
-      register(new PreFlexCodec());
-      register(new PulsingCodec());
-      register(new SepCodec());
-      codec = CodecProvider.CORE_CODECS[random.nextInt(CodecProvider.CORE_CODECS.length)];
-    }
-    
-    @Override
-    public Codec getWriter(SegmentWriteState state) {
-      return lookup(codec);
-    }
-  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Tue Jul 27 20:44:34 2010
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
@@ -139,7 +138,6 @@ public class TestAddIndexes extends Luce
 
     setUpDirs(dir, aux);
     IndexWriter writer = newWriter(dir, new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer()).setOpenMode(OpenMode.APPEND));
-
     writer.addIndexes(new Directory[] {aux});
 
     // Adds 10 docs, then replaces them with another 10

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestAtomicUpdate.java Tue Jul 27 20:44:34 2010
@@ -47,7 +47,7 @@ public class TestAtomicUpdate extends Lu
   private static abstract class TimedThread extends Thread {
     volatile boolean failed;
     int count;
-    private static float RUN_TIME_SEC = 0.5f * (float)_TestUtil.getRandomMultiplier();
+    private static float RUN_TIME_SEC = 0.5f * RANDOM_MULTIPLIER;
     private TimedThread[] allThreads;
 
     abstract public void doWork() throws Throwable;

Propchange: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
------------------------------------------------------------------------------
--- svn:mergeinfo (original)
+++ svn:mergeinfo Tue Jul 27 20:44:34 2010
@@ -1,5 +1,6 @@
-/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:943137,949730,957490,960490,961612
-/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:953476-978809
+/lucene/dev/branches/branch_3x/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:943137,949730,957490,960490,961612,979161
+/lucene/dev/branches/preflexfixes/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:967125-979432
+/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:953476-979858
 /lucene/java/branches/flex_1458/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:824912-931101
 /lucene/java/branches/lucene_2_4/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:748824
 /lucene/java/branches/lucene_2_9/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java:829134,829881,831036,896850,909334,948516

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestByteSlices.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestByteSlices.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestByteSlices.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestByteSlices.java Tue Jul 27 20:44:34 2010
@@ -18,7 +18,6 @@ import java.util.Random;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
 
 public class TestByteSlices extends LuceneTestCase {
 
@@ -55,7 +54,7 @@ public class TestByteSlices extends Luce
   public void testBasic() throws Throwable {
     ByteBlockPool pool = new ByteBlockPool(new ByteBlockAllocator());
 
-    final int NUM_STREAM = 100*_TestUtil.getRandomMultiplier();
+    final int NUM_STREAM = 100 * RANDOM_MULTIPLIER;
 
     ByteSliceWriter writer = new ByteSliceWriter(pool);
 
@@ -74,7 +73,8 @@ public class TestByteSlices extends Luce
         counters[stream] = 0;
       }
       
-      for(int iter=0;iter<10000*_TestUtil.getRandomMultiplier();iter++) {
+      int num = 10000 * RANDOM_MULTIPLIER;
+      for (int iter = 0; iter < num; iter++) {
         int stream = r.nextInt(NUM_STREAM);
         if (VERBOSE)
           System.out.println("write stream=" + stream);

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestCodecs.java Tue Jul 27 20:44:34 2010
@@ -44,7 +44,6 @@ import org.apache.lucene.store.RAMDirect
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.MultiCodecTestCase;
 import org.apache.lucene.util.Version;
-import org.apache.lucene.util._TestUtil;
 
 // TODO: test multiple codecs here?
 
@@ -69,7 +68,7 @@ public class TestCodecs extends MultiCod
   private Random RANDOM;
   private static String[] fieldNames = new String[] {"one", "two", "three", "four"};
 
-  private final static int NUM_TEST_ITER = 20*_TestUtil.getRandomMultiplier();
+  private final static int NUM_TEST_ITER = 20 * RANDOM_MULTIPLIER;
   private final static int NUM_TEST_THREADS = 3;
   private final static int NUM_FIELDS = 4;
   private final static int NUM_TERMS_RAND = 50; // must be > 16 to test skipping
@@ -493,14 +492,21 @@ public class TestCodecs extends MultiCod
         // Test random seek by ord:
         final int idx = TestCodecs.this.nextInt(field.terms.length);
         term = field.terms[idx];
-        status = termsEnum.seek(idx);
-        assertEquals(status, TermsEnum.SeekStatus.FOUND);
-        assertTrue(termsEnum.term().bytesEquals(new BytesRef(term.text2)));
-        assertEquals(term.docs.length, termsEnum.docFreq());
-        if (field.omitTF) {
-          this.verifyDocs(term.docs, term.positions, termsEnum.docs(null, null), false);
-        } else {
-          this.verifyDocs(term.docs, term.positions, termsEnum.docsAndPositions(null, null), true);
+        try {
+          status = termsEnum.seek(idx);
+        } catch (UnsupportedOperationException uoe) {
+          // ok -- skip it
+          status = null;
+        }
+        if (status != null) {
+          assertEquals(status, TermsEnum.SeekStatus.FOUND);
+          assertTrue(termsEnum.term().bytesEquals(new BytesRef(term.text2)));
+          assertEquals(term.docs.length, termsEnum.docFreq());
+          if (field.omitTF) {
+            this.verifyDocs(term.docs, term.positions, termsEnum.docs(null, null), false);
+          } else {
+            this.verifyDocs(term.docs, term.positions, termsEnum.docsAndPositions(null, null), true);
+          }
         }
 
         // Test seek to non-existent terms:
@@ -520,9 +526,12 @@ public class TestCodecs extends MultiCod
 
         // Seek to each term by ord, backwards
         for(int i=field.terms.length-1;i>=0;i--) {
-          assertEquals(Thread.currentThread().getName() + ": field=" + field.fieldInfo.name + " term=" + field.terms[i].text2, TermsEnum.SeekStatus.FOUND, termsEnum.seek(i));
-          assertEquals(field.terms[i].docs.length, termsEnum.docFreq());
-          assertTrue(termsEnum.term().bytesEquals(new BytesRef(field.terms[i].text2)));
+          try {
+            assertEquals(Thread.currentThread().getName() + ": field=" + field.fieldInfo.name + " term=" + field.terms[i].text2, TermsEnum.SeekStatus.FOUND, termsEnum.seek(i));
+            assertEquals(field.terms[i].docs.length, termsEnum.docFreq());
+            assertTrue(termsEnum.term().bytesEquals(new BytesRef(field.terms[i].text2)));
+          } catch (UnsupportedOperationException uoe) {
+          }
         }
 
         // Seek to non-existent empty-string term

Modified: lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestFlex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestFlex.java?rev=979860&r1=979859&r2=979860&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestFlex.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test/org/apache/lucene/index/TestFlex.java Tue Jul 27 20:44:34 2010
@@ -20,6 +20,8 @@ package org.apache.lucene.index;
 import java.io.*;
 import java.util.*;
 import org.apache.lucene.store.*;
+import org.apache.lucene.index.codecs.*;
+import org.apache.lucene.index.codecs.standard.*;
 import org.apache.lucene.search.*;
 import org.apache.lucene.analysis.*;
 import org.apache.lucene.document.*;
@@ -64,7 +66,8 @@ public class TestFlex extends LuceneTest
 
   public void testTermOrd() throws Exception {
     Directory d = new MockRAMDirectory();
-    IndexWriter w = new IndexWriter(d, new MockAnalyzer(), IndexWriter.MaxFieldLength.UNLIMITED);
+    IndexWriter w = new IndexWriter(d, new IndexWriterConfig(TEST_VERSION_CURRENT,
+                                                             new MockAnalyzer()).setCodecProvider(_TestUtil.alwaysCodec("Standard")));
     Document doc = new Document();
     doc.add(new Field("f", "a b c", Field.Store.NO, Field.Index.ANALYZED));
     w.addDocument(doc);