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/03/15 22:35:35 UTC

svn commit: r1081952 [7/17] - in /lucene/dev/branches/bulkpostings: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/demo/ dev-tools/idea/luce...

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MergePolicy.java Tue Mar 15 21:35:17 2011
@@ -72,8 +72,8 @@ public abstract class MergePolicy implem
     long mergeGen;                  // used by IndexWriter
     boolean isExternal;             // used by IndexWriter
     int maxNumSegmentsOptimize;     // used by IndexWriter
-    SegmentReader[] readers;        // used by IndexWriter
-    SegmentReader[] readersClone;   // used by IndexWriter
+    List<SegmentReader> readers;        // used by IndexWriter
+    List<SegmentReader> readerClones;   // used by IndexWriter
     public final SegmentInfos segments;
     boolean aborted;
     Throwable error;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFieldsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFieldsEnum.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFieldsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFieldsEnum.java Tue Mar 15 21:35:17 2011
@@ -129,7 +129,7 @@ public final  class MultiFieldsEnum exte
 
   private final static class FieldMergeQueue extends PriorityQueue<FieldsEnumWithSlice> {
     FieldMergeQueue(int size) {
-      initialize(size);
+      super(size);
     }
 
     @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java Tue Mar 15 21:35:17 2011
@@ -469,7 +469,7 @@ public final class MultiTermsEnum extend
   private final static class TermMergeQueue extends PriorityQueue<TermsEnumWithSlice> {
     Comparator<BytesRef> termComp;
     TermMergeQueue(int size) {
-      initialize(size);
+      super(size);
     }
 
     @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/OrdTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/OrdTermState.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/OrdTermState.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/OrdTermState.java Tue Mar 15 21:35:17 2011
@@ -30,4 +30,9 @@ public class OrdTermState extends TermSt
     assert other instanceof OrdTermState : "can not copy from " + other.getClass().getName();
     this.ord = ((OrdTermState) other).ord;
   }
+
+  @Override
+  public String toString() {
+    return "OrdTermState ord=" + ord;
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Tue Mar 15 21:35:17 2011
@@ -571,6 +571,11 @@ final class SegmentMerger {
     return mergeState.delCounts;
   }
   
+  public boolean getAnyNonBulkMerges() {
+    assert matchedCount <= readers.size();
+    return matchedCount != readers.size();
+  }
+
   private void mergeNorms() throws IOException {
     IndexOutput output = null;
     try {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java Tue Mar 15 21:35:17 2011
@@ -702,7 +702,7 @@ public class SegmentReader extends Index
     }
   }
 
-  private void commitChanges(Map<String,String> commitUserData) throws IOException {
+  private synchronized void commitChanges(Map<String,String> commitUserData) throws IOException {
     if (deletedDocsDirty) {               // re-write deleted
       si.advanceDelGen();
 
@@ -923,26 +923,21 @@ public class SegmentReader extends Index
     return fieldSet;
   }
 
-
   @Override
-  public synchronized boolean hasNorms(String field) {
+  public boolean hasNorms(String field) {
     ensureOpen();
     return norms.containsKey(field);
   }
 
-  // can return null if norms aren't stored
-  protected synchronized byte[] getNorms(String field) throws IOException {
-    Norm norm = norms.get(field);
-    if (norm == null) return null;  // not indexed, or norms not stored
-    return norm.bytes();
-  }
-
-  // returns fake norms if norms aren't available
   @Override
-  public synchronized byte[] norms(String field) throws IOException {
+  public byte[] norms(String field) throws IOException {
     ensureOpen();
-    byte[] bytes = getNorms(field);
-    return bytes;
+    final Norm norm = norms.get(field);
+    if (norm == null) {
+      // not indexed, or norms not stored
+      return null;  
+    }
+    return norm.bytes();
   }
 
   @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermState.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermState.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermState.java Tue Mar 15 21:35:17 2011
@@ -44,4 +44,9 @@ public abstract class TermState implemen
       throw new RuntimeException(cnse);
     }
   } 
-}
\ No newline at end of file
+
+  @Override
+  public String toString() {
+    return "TermState";
+  }
+}

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java Tue Mar 15 21:35:17 2011
@@ -51,6 +51,6 @@ public class BlockTermState extends OrdT
 
   @Override
   public String toString() {
-    return super.toString() + "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
+    return "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
   }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Tue Mar 15 21:35:17 2011
@@ -67,9 +67,6 @@ public class BlockTermsReader extends Fi
 
   private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
 
-  // Comparator that orders our terms
-  private final Comparator<BytesRef> termComp;
-
   // Caches the most recently looked-up field + terms:
   private final DoubleBarrelLRUCache<FieldAndTerm,BlockTermState> termsCache;
 
@@ -112,13 +109,12 @@ public class BlockTermsReader extends Fi
   //private String segment;
   
   public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
-                          Comparator<BytesRef> termComp, int termsCacheSize, String codecId)
+                          int termsCacheSize, String codecId)
     throws IOException {
     
     this.postingsReader = postingsReader;
     termsCache = new DoubleBarrelLRUCache<FieldAndTerm,BlockTermState>(termsCacheSize);
 
-    this.termComp = termComp;
     //this.segment = segment;
     in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
                        readBufferSize);
@@ -261,7 +257,7 @@ public class BlockTermsReader extends Fi
 
     @Override
     public Comparator<BytesRef> getComparator() {
-      return termComp;
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
     }
 
     @Override
@@ -343,23 +339,29 @@ public class BlockTermsReader extends Fi
 
       @Override
       public Comparator<BytesRef> getComparator() {
-        return termComp;
+        return BytesRef.getUTF8SortedAsUnicodeComparator();
       }
 
+      // TODO: we may want an alternate mode here which is
+      // "if you are about to return NOT_FOUND I won't use
+      // the terms data from that"; eg FuzzyTermsEnum will
+      // (usually) just immediately call seek again if we
+      // return NOT_FOUND so it's a waste for us to fill in
+      // the term that was actually NOT_FOUND
       @Override
       public SeekStatus seek(final BytesRef target, final boolean useCache) throws IOException {
 
         if (indexEnum == null) {
           throw new IllegalStateException("terms index was not loaded");
         }
-        
-        //System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this="  + this);
+   
         /*
+        System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this="  + this);
         if (didIndexNext) {
           if (nextIndexTerm == null) {
-            //System.out.println("  nextIndexTerm=null");
+            System.out.println("  nextIndexTerm=null");
           } else {
-            //System.out.println("  nextIndexTerm=" + nextIndexTerm.utf8ToString());
+            System.out.println("  nextIndexTerm=" + nextIndexTerm.utf8ToString());
           }
         }
         */
@@ -387,7 +389,7 @@ public class BlockTermsReader extends Fi
         // is after current term but before next index term:
         if (indexIsCurrent) {
 
-          final int cmp = termComp.compare(term, target);
+          final int cmp = BytesRef.getUTF8SortedAsUnicodeComparator().compare(term, target);
 
           if (cmp == 0) {
             // Already at the requested term
@@ -405,7 +407,7 @@ public class BlockTermsReader extends Fi
               didIndexNext = true;
             }
 
-            if (nextIndexTerm == null || termComp.compare(target, nextIndexTerm) < 0) {
+            if (nextIndexTerm == null || BytesRef.getUTF8SortedAsUnicodeComparator().compare(target, nextIndexTerm) < 0) {
               // Optimization: requested term is within the
               // same term block we are now in; skip seeking
               // (but do scanning):
@@ -435,48 +437,175 @@ public class BlockTermsReader extends Fi
             state.ord = indexEnum.ord()-1;
           }
 
-          // NOTE: the first _next() after an index seek is
-          // a bit wasteful, since it redundantly reads some
-          // suffix bytes into the buffer.  We could avoid storing
-          // those bytes in the primary file, but then when
-          // next()ing over an index term we'd have to
-          // special case it:
           term.copy(indexEnum.term());
           //System.out.println("  seek: term=" + term.utf8ToString());
         } else {
-          ////System.out.println("  skip seek");
+          //System.out.println("  skip seek");
+          if (state.termCount == state.blockTermCount && !nextBlock()) {
+            indexIsCurrent = false;
+            return SeekStatus.END;
+          }
         }
 
         seekPending = false;
 
-        // Now scan:
-        while (_next() != null) {
-          final int cmp = termComp.compare(term, target);
-          if (cmp == 0) {
-            // Match!
-            if (useCache) {
-              // Store in cache
-              decodeMetaData();
-              termsCache.put(new FieldAndTerm(fieldTerm), (BlockTermState) state.clone());
+        int common = 0;
+
+        // Scan within block.  We could do this by calling
+        // _next() and testing the resulting term, but this
+        // is wasteful.  Instead, we first confirm the
+        // target matches the common prefix of this block,
+        // and then we scan the term bytes directly from the
+        // termSuffixesreader's byte[], saving a copy into
+        // the BytesRef term per term.  Only when we return
+        // do we then copy the bytes into the term.
+
+        while(true) {
+
+          // First, see if target term matches common prefix
+          // in this block:
+          if (common < termBlockPrefix) {
+            final int cmp = (term.bytes[common]&0xFF) - (target.bytes[target.offset + common]&0xFF);
+            if (cmp < 0) {
+
+              // TODO: maybe we should store common prefix
+              // in block header?  (instead of relying on
+              // last term of previous block)
+
+              // Target's prefix is after the common block
+              // prefix, so term cannot be in this block
+              // but it could be in next block.  We
+              // must scan to end-of-block to set common
+              // prefix for next block:
+              if (state.termCount < state.blockTermCount) {
+                while(state.termCount < state.blockTermCount-1) {
+                  state.termCount++;
+                  state.ord++;
+                  termSuffixesReader.skipBytes(termSuffixesReader.readVInt());
+                }
+                final int suffix = termSuffixesReader.readVInt();
+                term.length = termBlockPrefix + suffix;
+                if (term.bytes.length < term.length) {
+                  term.grow(term.length);
+                }
+                termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+              }
+              state.ord++;
+              
+              if (!nextBlock()) {
+                indexIsCurrent = false;
+                return SeekStatus.END;
+              }
+              common = 0;
+
+            } else if (cmp > 0) {
+              // Target's prefix is before the common prefix
+              // of this block, so we position to start of
+              // block and return NOT_FOUND:
+              assert state.termCount == 0;
+
+              final int suffix = termSuffixesReader.readVInt();
+              term.length = termBlockPrefix + suffix;
+              if (term.bytes.length < term.length) {
+                term.grow(term.length);
+              }
+              termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+              return SeekStatus.NOT_FOUND;
+            } else {
+              common++;
+            }
+
+            continue;
+          }
+
+          // Test every term in this block
+          while (true) {
+            state.termCount++;
+            state.ord++;
+
+            final int suffix = termSuffixesReader.readVInt();
+            
+            // We know the prefix matches, so just compare the new suffix:
+            final int termLen = termBlockPrefix + suffix;
+            int bytePos = termSuffixesReader.getPosition();
+
+            boolean next = false;
+            final int limit = target.offset + (termLen < target.length ? termLen : target.length);
+            int targetPos = target.offset + termBlockPrefix;
+            while(targetPos < limit) {
+              final int cmp = (termSuffixes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
+              if (cmp < 0) {
+                // Current term is still before the target;
+                // keep scanning
+                next = true;
+                break;
+              } else if (cmp > 0) {
+                // Done!  Current term is after target. Stop
+                // here, fill in real term, return NOT_FOUND.
+                term.length = termBlockPrefix + suffix;
+                if (term.bytes.length < term.length) {
+                  term.grow(term.length);
+                }
+                termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+                //System.out.println("  NOT_FOUND");
+                return SeekStatus.NOT_FOUND;
+              }
+            }
+
+            if (!next && target.length <= termLen) {
+              term.length = termBlockPrefix + suffix;
+              if (term.bytes.length < term.length) {
+                term.grow(term.length);
+              }
+              termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+
+              if (target.length == termLen) {
+                // Done!  Exact match.  Stop here, fill in
+                // real term, return FOUND.
+                //System.out.println("  FOUND");
+
+                if (useCache) {
+                  // Store in cache
+                  decodeMetaData();
+                  //System.out.println("  cache! state=" + state);
+                  termsCache.put(new FieldAndTerm(fieldTerm), (BlockTermState) state.clone());
+                }
+
+                return SeekStatus.FOUND;
+              } else {
+                //System.out.println("  NOT_FOUND");
+                return SeekStatus.NOT_FOUND;
+              }
+            }
+
+            if (state.termCount == state.blockTermCount) {
+              // Must pre-fill term for next block's common prefix
+              term.length = termBlockPrefix + suffix;
+              if (term.bytes.length < term.length) {
+                term.grow(term.length);
+              }
+              termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+              break;
+            } else {
+              termSuffixesReader.skipBytes(suffix);
             }
-            //System.out.println("  FOUND");
-            return SeekStatus.FOUND;
-          } else if (cmp > 0) {
-            //System.out.println("  NOT_FOUND term=" + term.utf8ToString());
-            return SeekStatus.NOT_FOUND;
           }
-          
+
           // The purpose of the terms dict index is to seek
           // the enum to the closest index term before the
           // term we are looking for.  So, we should never
           // cross another index term (besides the first
           // one) while we are scanning:
+
           assert indexIsCurrent;
-        }
 
-        indexIsCurrent = false;
-        //System.out.println("  END");
-        return SeekStatus.END;
+          if (!nextBlock()) {
+            //System.out.println("  END");
+            indexIsCurrent = false;
+            return SeekStatus.END;
+          }
+          common = 0;
+        }
       }
 
       @Override
@@ -516,12 +645,10 @@ public class BlockTermsReader extends Fi
          decode all metadata up to the current term. */
       private BytesRef _next() throws IOException {
         //System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termCount + " (vs " + state.blockTermCount + ")");
-        if (state.termCount == state.blockTermCount) {
-          if (!nextBlock()) {
-            //System.out.println("  eof");
-            indexIsCurrent = false;
-            return null;
-          }
+        if (state.termCount == state.blockTermCount && !nextBlock()) {
+          //System.out.println("  eof");
+          indexIsCurrent = false;
+          return null;
         }
 
         // TODO: cutover to something better for these ints!  simple64?
@@ -697,7 +824,7 @@ public class BlockTermsReader extends Fi
         }
         //System.out.println("  termSuffixes len=" + len);
         in.readBytes(termSuffixes, 0, len);
-        termSuffixesReader.reset(termSuffixes);
+        termSuffixesReader.reset(termSuffixes, 0, len);
 
         // docFreq, totalTermFreq
         len = in.readVInt();
@@ -706,7 +833,7 @@ public class BlockTermsReader extends Fi
         }
         //System.out.println("  freq bytes len=" + len);
         in.readBytes(docFreqBytes, 0, len);
-        freqReader.reset(docFreqBytes);
+        freqReader.reset(docFreqBytes, 0, len);
         metaDataUpto = 0;
 
         state.termCount = 0;
@@ -725,23 +852,32 @@ public class BlockTermsReader extends Fi
         if (!seekPending) {
           // lazily catch up on metadata decode:
           final int limit = state.termCount;
+          // We must set/incr state.termCount because
+          // postings impl can look at this
           state.termCount = metaDataUpto;
+          // TODO: better API would be "jump straight to term=N"???
           while (metaDataUpto < limit) {
-            //System.out.println("  decode");
+            //System.out.println("  decode mdUpto=" + metaDataUpto);
             // TODO: we could make "tiers" of metadata, ie,
             // decode docFreq/totalTF but don't decode postings
             // metadata; this way caller could get
             // docFreq/totalTF w/o paying decode cost for
             // postings
+
+            // TODO: if docFreq were bulk decoded we could
+            // just skipN here:
             state.docFreq = freqReader.readVInt();
+            //System.out.println("    dF=" + state.docFreq);
             if (!fieldInfo.omitTermFreqAndPositions) {
               state.totalTermFreq = state.docFreq + freqReader.readVLong();
+              //System.out.println("    totTF=" + state.totalTermFreq);
             }
+
             postingsReader.nextTerm(fieldInfo, state);
             metaDataUpto++;
             state.termCount++;
           }
-        } else {
+        //} else {
           //System.out.println("  skip! seekPending");
         }
       }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java Tue Mar 15 21:35:17 2011
@@ -63,24 +63,23 @@ public class BlockTermsWriter extends Fi
   FieldInfo currentField;
   private final TermsIndexWriterBase termsIndexWriter;
   private final List<TermsWriter> fields = new ArrayList<TermsWriter>();
-  private final Comparator<BytesRef> termComp;
-  private final String segment;
+
+  //private final String segment;
 
   public BlockTermsWriter(
       TermsIndexWriterBase termsIndexWriter,
       SegmentWriteState state,
-      PostingsWriterBase postingsWriter,
-      Comparator<BytesRef> termComp) throws IOException
+      PostingsWriterBase postingsWriter)
+    throws IOException
   {
     final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
     this.termsIndexWriter = termsIndexWriter;
-    this.termComp = termComp;
     out = state.directory.createOutput(termsFileName);
     fieldInfos = state.fieldInfos;
     writeHeader(out);
     currentField = null;
     this.postingsWriter = postingsWriter;
-    segment = state.segmentName;
+    //segment = state.segmentName;
 
     //System.out.println("BTW.init seg=" + state.segmentName);
 
@@ -161,7 +160,6 @@ public class BlockTermsWriter extends Fi
     private long numTerms;
     private final TermsIndexWriterBase.FieldWriter fieldIndexWriter;
     long sumTotalTermFreq;
-    private final BytesRef lastTerm = new BytesRef();
 
     private TermEntry[] pendingTerms;
 
@@ -185,12 +183,12 @@ public class BlockTermsWriter extends Fi
     
     @Override
     public Comparator<BytesRef> getComparator() {
-      return termComp;
+      return BytesRef.getUTF8SortedAsUnicodeComparator();
     }
 
     @Override
     public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      //System.out.println("BTW.startTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text);
+      //System.out.println("BTW.startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
       postingsWriter.startTerm();
       return postingsWriter;
     }
@@ -201,7 +199,7 @@ public class BlockTermsWriter extends Fi
     public void finishTerm(BytesRef text, TermStats stats) throws IOException {
 
       assert stats.docFreq > 0;
-      //System.out.println("BTW.finishTerm seg=" + segment + " term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " df=" + stats.docFreq);
+      //System.out.println("BTW.finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
 
       final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
 
@@ -213,6 +211,7 @@ public class BlockTermsWriter extends Fi
           flushBlock();
         }
         fieldIndexWriter.add(text, stats, out.getFilePointer());
+        //System.out.println("  index term!");
       }
 
       if (pendingTerms.length == pendingCount) {
@@ -265,7 +264,7 @@ public class BlockTermsWriter extends Fi
     private final RAMOutputStream bytesWriter = new RAMOutputStream();
 
     private void flushBlock() throws IOException {
-      //System.out.println("BTW.flushBlock pendingCount=" + pendingCount);
+      //System.out.println("BTW.flushBlock seg=" + segment + " pendingCount=" + pendingCount + " fp=" + out.getFilePointer());
 
       // First pass: compute common prefix for all terms
       // in the block, against term before first term in

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/bulkvint/BulkVIntCodec.java Tue Mar 15 21:35:17 2011
@@ -167,7 +167,7 @@ public class BulkVIntCodec extends Codec
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -211,7 +211,6 @@ public class BulkVIntCodec extends Codec
                                                 state.segmentInfo.name,
                                                 postingsReader,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 StandardCodec.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/FrameOfRefCodec.java Tue Mar 15 21:35:17 2011
@@ -63,7 +63,7 @@ public class FrameOfRefCodec extends Cod
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -108,7 +108,6 @@ public class FrameOfRefCodec extends Cod
                                                       state.segmentInfo.name,
                                                       postingsReader,
                                                       state.readBufferSize,
-                                                      BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                       StandardCodec.TERMS_CACHE_SIZE,
                                                       state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta/PatchedFrameOfRefCodec.java Tue Mar 15 21:35:17 2011
@@ -63,7 +63,7 @@ public class PatchedFrameOfRefCodec exte
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -108,7 +108,6 @@ public class PatchedFrameOfRefCodec exte
                                                       state.segmentInfo.name,
                                                       postingsReader,
                                                       state.readBufferSize,
-                                                      BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                       StandardCodec.TERMS_CACHE_SIZE,
                                                       state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pfordelta2/PForDeltaFixedIntBlockCodec.java Tue Mar 15 21:35:17 2011
@@ -183,7 +183,7 @@ public class PForDeltaFixedIntBlockCodec
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -227,7 +227,6 @@ public class PForDeltaFixedIntBlockCodec
                                                        state.segmentInfo.name,
                                                        postingsReader,
                                                        state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                        StandardCodec.TERMS_CACHE_SIZE,
                                                        state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Tue Mar 15 21:35:17 2011
@@ -30,15 +30,14 @@ import org.apache.lucene.index.codecs.Po
 import org.apache.lucene.index.codecs.standard.StandardPostingsReader;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
-import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
-import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
+import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
+import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.BlockTermsReader;
 import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BytesRef;
 
 /** This codec "inlines" the postings for terms that have
  *  low docFreq.  It wraps another codec, which is used for
@@ -78,7 +77,7 @@ public class PulsingCodec extends Codec 
     TermsIndexWriterBase indexWriter;
     boolean success = false;
     try {
-      indexWriter = new FixedGapTermsIndexWriter(state);
+      indexWriter = new VariableGapTermsIndexWriter(state, new VariableGapTermsIndexWriter.EveryNTermSelector(state.termIndexInterval));
       success = true;
     } finally {
       if (!success) {
@@ -89,7 +88,7 @@ public class PulsingCodec extends Codec 
     // Terms dict
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter);
       success = true;
       return ret;
     } finally {
@@ -116,12 +115,11 @@ public class PulsingCodec extends Codec 
 
     boolean success = false;
     try {
-      indexReader = new FixedGapTermsIndexReader(state.dir,
-                                                       state.fieldInfos,
-                                                       state.segmentInfo.name,
-                                                       state.termsIndexDivisor,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       state.codecId);
+      indexReader = new VariableGapTermsIndexReader(state.dir,
+                                                    state.fieldInfos,
+                                                    state.segmentInfo.name,
+                                                    state.termsIndexDivisor,
+                                                    state.codecId);
       success = true;
     } finally {
       if (!success) {
@@ -136,7 +134,6 @@ public class PulsingCodec extends Codec 
                                                 state.dir, state.fieldInfos, state.segmentInfo.name,
                                                 pulsingReader,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 StandardCodec.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
@@ -156,7 +153,7 @@ public class PulsingCodec extends Codec 
   public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
     StandardPostingsReader.files(dir, segmentInfo, id, files);
     BlockTermsReader.files(dir, segmentInfo, id, files);
-    FixedGapTermsIndexReader.files(dir, segmentInfo, id, files);
+    VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
   }
 
   @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Tue Mar 15 21:35:17 2011
@@ -145,7 +145,7 @@ public class PulsingPostingsReaderImpl e
     //System.out.println("  count=" + count + " threshold=" + maxPositions);
 
     if (count <= maxPositions) {
-      //System.out.println("  inlined");
+      //System.out.println("  inlined pos=" + termState.inlinedBytesReader.getPosition());
 
       // Inlined into terms dict -- just read the byte[] blob in,
       // but don't decode it now (we only decode when a DocsEnum

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/simple64/Simple64Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/simple64/Simple64Codec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/simple64/Simple64Codec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/simple64/Simple64Codec.java Tue Mar 15 21:35:17 2011
@@ -150,7 +150,7 @@ public class Simple64Codec extends Codec
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, postingsWriter);
       success = true;
       return ret;
     } finally {
@@ -194,7 +194,6 @@ public class Simple64Codec extends Codec
                                                        state.segmentInfo.name,
                                                        postingsReader,
                                                        state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                        StandardCodec.TERMS_CACHE_SIZE,
                                                        state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Tue Mar 15 21:35:17 2011
@@ -23,7 +23,6 @@ import java.util.Set;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
-import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
@@ -66,7 +65,7 @@ public class StandardCodec extends Codec
 
     success = false;
     try {
-      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs);
       success = true;
       return ret;
     } finally {
@@ -109,7 +108,6 @@ public class StandardCodec extends Codec
                                                 state.segmentInfo.name,
                                                 postings,
                                                 state.readBufferSize,
-                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
                                                 TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/messages/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/messages/package.html?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/messages/package.html (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/messages/package.html Tue Mar 15 21:35:17 2011
@@ -45,7 +45,7 @@ Features:
 <p>
 Lazy loading of Message Strings
 
-<pre>
+<pre class="prettyprint">
 	public class MessagesTestBundle extends NLS {
 	
 	  private static final String BUNDLE_NAME = MessagesTestBundle.class.getName();
@@ -85,7 +85,7 @@ Lazy loading of Message Strings
 <p>
 Normal loading of Message Strings
 
-<pre>
+<pre class="prettyprint">
 	String message1 = NLS.getLocalizedMessage(MessagesTestBundle.Q0004E_INVALID_SYNTAX_ESCAPE_UNICODE_TRUNCATION);
 	String message2 = NLS.getLocalizedMessage(MessagesTestBundle.Q0004E_INVALID_SYNTAX_ESCAPE_UNICODE_TRUNCATION, Locale.JAPANESE);
 </pre>

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java Tue Mar 15 21:35:17 2011
@@ -78,9 +78,9 @@ public abstract class QueryParserBase {
   // maps field names to date resolutions
   Map<String,DateTools.Resolution> fieldToDateResolution = null;
 
-  // The collator to use when determining range inclusion,
-  // for use when constructing RangeQuerys.
-  Collator rangeCollator = null;
+  //Whether or not to analyze range terms when constructing RangeQuerys
+  // (For example, analyzing terms into collation keys for locale-sensitive RangeQuery)
+  boolean analyzeRangeTerms = false;
 
   boolean autoGeneratePhraseQueries;
 
@@ -391,27 +391,21 @@ public abstract class QueryParserBase {
   }
 
   /**
-   * Sets the collator used to determine index term inclusion in ranges
-   * for RangeQuerys.
-   * <p/>
-   * <strong>WARNING:</strong> Setting the rangeCollator to a non-null
-   * collator using this method will cause every single index Term in the
-   * Field referenced by lowerTerm and/or upperTerm to be examined.
-   * Depending on the number of index Terms in this Field, the operation could
-   * be very slow.
-   *
-   *  @param rc  the collator to use when constructing RangeQuerys
+   * Set whether or not to analyze range terms when constructing RangeQuerys.
+   * For example, setting this to true can enable analyzing terms into 
+   * collation keys for locale-sensitive RangeQuery.
+   * 
+   * @param analyzeRangeTerms whether or not terms should be analyzed for RangeQuerys
    */
-  public void setRangeCollator(Collator rc) {
-    rangeCollator = rc;
+  public void setAnalyzeRangeTerms(boolean analyzeRangeTerms) {
+    this.analyzeRangeTerms = analyzeRangeTerms;
   }
 
   /**
-   * @return the collator used to determine index term inclusion in ranges
-   * for RangeQuerys.
+   * @return whether or not to analyze range terms when constructing RangeQuerys.
    */
-  public Collator getRangeCollator() {
-    return rangeCollator;
+  public boolean getAnalyzeRangeTerms() {
+    return analyzeRangeTerms;
   }
 
   protected void addClause(List<BooleanClause> clauses, int conj, int mods, Query q) {
@@ -792,6 +786,36 @@ public abstract class QueryParserBase {
     return new FuzzyQuery(term,minimumSimilarity,prefixLength);
   }
 
+  private BytesRef analyzeRangePart(String field, String part) {
+    TokenStream source;
+      
+    try {
+      source = analyzer.reusableTokenStream(field, new StringReader(part));
+      source.reset();
+    } catch (IOException e) {
+      source = analyzer.tokenStream(field, new StringReader(part));
+    }
+      
+    BytesRef result = new BytesRef();
+    TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
+      
+    try {
+      if (!source.incrementToken())
+        throw new IllegalArgumentException("analyzer returned no terms for range part: " + part);
+      termAtt.toBytesRef(result);
+      if (source.incrementToken())
+        throw new IllegalArgumentException("analyzer returned too many terms for range part: " + part);
+    } catch (IOException e) {
+      throw new RuntimeException("error analyzing range part: " + part, e);
+    }
+      
+    try {
+      source.close();
+    } catch (IOException ignored) {}
+      
+    return result;
+  }
+
   /**
    * Builds a new TermRangeQuery instance
    * @param field Field
@@ -802,7 +826,23 @@ public abstract class QueryParserBase {
    * @return new TermRangeQuery instance
    */
   protected Query newRangeQuery(String field, String part1, String part2, boolean startInclusive, boolean endInclusive) {
-    final TermRangeQuery query = new TermRangeQuery(field, part1, part2, startInclusive, endInclusive, rangeCollator);
+    final BytesRef start;
+    final BytesRef end;
+     
+    if (part1 == null) {
+      start = null;
+    } else {
+      start = analyzeRangeTerms ? analyzeRangePart(field, part1) : new BytesRef(part1);
+    }
+     
+    if (part2 == null) {
+      end = null;
+    } else {
+      end = analyzeRangeTerms ? analyzeRangePart(field, part2) : new BytesRef(part2);
+    }
+      
+    final TermRangeQuery query = new TermRangeQuery(field, start, end, startInclusive, endInclusive);
+
     query.setRewriteMethod(multiTermRewriteMethod);
     return query;
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldComparator.java Tue Mar 15 21:35:17 2011
@@ -18,8 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.text.Collator;
-import java.util.Locale;
 
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.FieldCache.DocTermsIndex;
@@ -718,85 +716,6 @@ public abstract class FieldComparator {
     }
   }
   
-  
-  /** Sorts by a field's value using the Collator for a
-   *  given Locale.
-   *
-   * <p><b>WARNING</b>: this is likely very slow; you'll
-   * get much better performance using the
-   * CollationKeyAnalyzer or ICUCollationKeyAnalyzer. */
-  public static final class StringComparatorLocale extends FieldComparator {
-
-    private final String[] values;
-    private DocTerms currentDocTerms;
-    private final String field;
-    final Collator collator;
-    private String bottom;
-    private final BytesRef tempBR = new BytesRef();
-
-    StringComparatorLocale(int numHits, String field, Locale locale) {
-      values = new String[numHits];
-      this.field = field;
-      collator = Collator.getInstance(locale);
-    }
-
-    @Override
-    public int compare(int slot1, int slot2) {
-      final String val1 = values[slot1];
-      final String val2 = values[slot2];
-      if (val1 == null) {
-        if (val2 == null) {
-          return 0;
-        }
-        return -1;
-      } else if (val2 == null) {
-        return 1;
-      }
-      return collator.compare(val1, val2);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      final String val2 = currentDocTerms.getTerm(doc, tempBR).utf8ToString();
-      if (bottom == null) {
-        if (val2 == null) {
-          return 0;
-        }
-        return -1;
-      } else if (val2 == null) {
-        return 1;
-      }
-      return collator.compare(bottom, val2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      final BytesRef br = currentDocTerms.getTerm(doc, tempBR);
-      if (br == null) {
-        values[slot] = null;
-      } else {
-        values[slot] = br.utf8ToString();
-      }
-    }
-
-    @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
-      currentDocTerms = FieldCache.DEFAULT.getTerms(context.reader, field);
-      return this;
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public Comparable<?> value(int slot) {
-      final String s = values[slot];
-      return s == null ? null : new BytesRef(values[slot]);
-    }
-  }
-
   /** Sorts by field's natural Term sort order, using
    *  ordinals.  This is functionally equivalent to {@link
    *  TermValComparator}, but it first resolves the string

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java Tue Mar 15 21:35:17 2011
@@ -56,15 +56,13 @@ public abstract class FieldValueHitQueue
     
     public OneComparatorFieldValueHitQueue(SortField[] fields, int size)
         throws IOException {
-      super(fields);
+      super(fields, size);
 
       SortField field = fields[0];
       setComparator(0,field.getComparator(size, 0));
       oneReverseMul = field.reverse ? -1 : 1;
 
       reverseMul[0] = oneReverseMul;
-      
-      initialize(size);
     }
 
     /**
@@ -98,7 +96,7 @@ public abstract class FieldValueHitQueue
 
     public MultiComparatorsFieldValueHitQueue(SortField[] fields, int size)
         throws IOException {
-      super(fields);
+      super(fields, size);
 
       int numComparators = comparators.length;
       for (int i = 0; i < numComparators; ++i) {
@@ -107,8 +105,6 @@ public abstract class FieldValueHitQueue
         reverseMul[i] = field.reverse ? -1 : 1;
         setComparator(i, field.getComparator(size, i));
       }
-
-      initialize(size);
     }
   
     @Override
@@ -133,7 +129,8 @@ public abstract class FieldValueHitQueue
   }
   
   // prevent instantiation and extension.
-  private FieldValueHitQueue(SortField[] fields) {
+  private FieldValueHitQueue(SortField[] fields, int size) {
+    super(size);
     // When we get here, fields.length is guaranteed to be > 0, therefore no
     // need to check it again.
     

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/HitQueue.java Tue Mar 15 21:35:17 2011
@@ -63,17 +63,15 @@ final class HitQueue extends PriorityQue
    * @see #getSentinelObject()
    */
   HitQueue(int size, boolean prePopulate) {
-    this.prePopulate = prePopulate;
-    initialize(size);
+    super(size, prePopulate);
   }
 
-  // Returns null if prePopulate is false.
   @Override
   protected ScoreDoc getSentinelObject() {
     // Always set the doc Id to MAX_VALUE so that it won't be favored by
     // lessThan. This generally should not happen since if score is not NEG_INF,
     // TopScoreDocCollector will always add the object to the queue.
-    return !prePopulate ? null : new ScoreDoc(Integer.MAX_VALUE, Float.NEGATIVE_INFINITY);
+    return new ScoreDoc(Integer.MAX_VALUE, Float.NEGATIVE_INFINITY);
   }
   
   @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Tue Mar 15 21:35:17 2011
@@ -429,7 +429,7 @@ public class IndexSearcher {
    * <p>NOTE: this does not compute scores by default.  If you
    * need scores, create a {@link TopFieldCollector}
    * instance by calling {@link TopFieldCollector#create} and
-   * then pass that to {@link #search(Weight, Filter,
+   * then pass that to {@link #search(IndexReader.AtomicReaderContext[], Weight, Filter,
    * Collector)}.</p>
    */
   protected TopFieldDocs search(Weight weight, Filter filter, int nDocs,
@@ -442,13 +442,17 @@ public class IndexSearcher {
       // use all leaves here!
       return search (leafContexts, weight, filter, nDocs, sort, fillFields);
     } else {
-      // TODO: make this respect fillFields
-      final FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue(nDocs);
+      final TopFieldCollector topCollector = TopFieldCollector.create(sort, nDocs,
+                                                                      fillFields,
+                                                                      fieldSortDoTrackScores,
+                                                                      fieldSortDoMaxScore,
+                                                                      false);
+
       final Lock lock = new ReentrantLock();
       final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<TopFieldDocs>(executor);
       for (int i = 0; i < leafSlices.length; i++) { // search each leaf slice
         runner.submit(
-                      new SearcherCallableWithSort(lock, this, leafSlices[i], weight, filter, nDocs, hq, sort));
+                      new SearcherCallableWithSort(lock, this, leafSlices[i], weight, filter, nDocs, topCollector, sort));
       }
       int totalHits = 0;
       float maxScore = Float.NEGATIVE_INFINITY;
@@ -458,11 +462,10 @@ public class IndexSearcher {
           maxScore = Math.max(maxScore, topFieldDocs.getMaxScore());
         }
       }
-      final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
-      for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
-        scoreDocs[i] = hq.pop();
 
-      return new TopFieldDocs(totalHits, scoreDocs, hq.getFields(), maxScore);
+      final TopFieldDocs topDocs = (TopFieldDocs) topCollector.topDocs();
+
+      return new TopFieldDocs(totalHits, topDocs.scoreDocs, topDocs.fields, topDocs.getMaxScore());
     }
   }
   
@@ -475,7 +478,7 @@ public class IndexSearcher {
    * <p>NOTE: this does not compute scores by default.  If you
    * need scores, create a {@link TopFieldCollector}
    * instance by calling {@link TopFieldCollector#create} and
-   * then pass that to {@link #search(Weight, Filter,
+   * then pass that to {@link #search(IndexReader.AtomicReaderContext[], Weight, Filter,
    * Collector)}.</p>
    */
   protected TopFieldDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, int nDocs,
@@ -721,12 +724,12 @@ public class IndexSearcher {
     private final Weight weight;
     private final Filter filter;
     private final int nDocs;
-    private final FieldDocSortedHitQueue hq;
+    private final TopFieldCollector hq;
     private final Sort sort;
     private final LeafSlice slice;
 
     public SearcherCallableWithSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
-        Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort) {
+        Filter filter, int nDocs, TopFieldCollector hq, Sort sort) {
       this.lock = lock;
       this.searcher = searcher;
       this.weight = weight;
@@ -737,27 +740,58 @@ public class IndexSearcher {
       this.slice = slice;
     }
 
+    private final class FakeScorer extends Scorer {
+      float score;
+      int doc;
+
+      public FakeScorer() {
+        super(null);
+      }
+    
+      @Override
+      public int advance(int target) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int docID() {
+        return doc;
+      }
+
+      @Override
+      public float freq() {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
+      public int nextDoc() {
+        throw new UnsupportedOperationException();
+      }
+    
+      @Override
+      public float score() {
+        return score;
+      }
+    }
+
+    private final FakeScorer fakeScorer = new FakeScorer();
+
     public TopFieldDocs call() throws IOException {
+      assert slice.leaves.length == 1;
       final TopFieldDocs docs = searcher.search (slice.leaves, weight, filter, nDocs, sort, true);
       lock.lock();
       try {
-        hq.setFields(docs.fields);
+        final int base = slice.leaves[0].docBase;
+        hq.setNextReader(slice.leaves[0]);
+        hq.setScorer(fakeScorer);
+        for(ScoreDoc scoreDoc : docs.scoreDocs) {
+          fakeScorer.doc = scoreDoc.doc - base;
+          fakeScorer.score = scoreDoc.score;
+          hq.collect(scoreDoc.doc-base);
+        }
       } finally {
         lock.unlock();
       }
-
-      final ScoreDoc[] scoreDocs = docs.scoreDocs;
-      for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
-        final FieldDoc fieldDoc = (FieldDoc) scoreDocs[j];
-        //it would be so nice if we had a thread-safe insert 
-        lock.lock();
-        try {
-          if (fieldDoc == hq.insertWithOverflow(fieldDoc))
-            break;
-        } finally {
-          lock.unlock();
-        }
-      }
       return docs;
     }
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Tue Mar 15 21:35:17 2011
@@ -432,7 +432,7 @@ class UnionDocsAndPositionsEnum extends 
 
   private static final class DocsQueue extends PriorityQueue<DocsAndPositionsEnum> {
     DocsQueue(List<DocsAndPositionsEnum> docsEnums) throws IOException {
-      initialize(docsEnums.size());
+      super(docsEnums.size());
 
       Iterator<DocsAndPositionsEnum> i = docsEnums.iterator();
       while (i.hasNext()) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQuery.java Tue Mar 15 21:35:17 2011
@@ -224,7 +224,7 @@ public class PhraseQuery extends Query {
     public Explanation explain(AtomicReaderContext context, int doc)
       throws IOException {
 
-      Explanation result = new Explanation();
+      ComplexExplanation result = new ComplexExplanation();
       result.setDescription("weight("+getQuery()+" in "+doc+"), product of:");
 
       StringBuilder docFreqs = new StringBuilder();
@@ -303,10 +303,7 @@ public class PhraseQuery extends Query {
 
       // combine them
       result.setValue(queryExpl.getValue() * fieldExpl.getValue());
-
-      if (queryExpl.getValue() == 1.0f)
-        return fieldExpl;
-
+      result.setMatch(tfExplanation.isMatch());
       return result;
     }
   }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/PhraseQueue.java Tue Mar 15 21:35:17 2011
@@ -21,7 +21,7 @@ import org.apache.lucene.util.PriorityQu
 
 final class PhraseQueue extends PriorityQueue<PhrasePositions> {
   PhraseQueue(int size) {
-    initialize(size);
+    super(size);
   }
 
   @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/SortField.java Tue Mar 15 21:35:17 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Locale;
 
 import org.apache.lucene.search.cache.*;
 import org.apache.lucene.util.StringHelper;
@@ -90,7 +89,6 @@ public class SortField {
 
   private String field;
   private int type;  // defaults to determining type dynamically
-  private Locale locale;    // defaults to "natural order" (no Locale)
   boolean reverse = false;  // defaults to natural order
   private CachedArrayCreator<?> creator;
   public Object missingValue = null; // used for 'sortMissingFirst/Last'
@@ -213,28 +211,6 @@ public class SortField {
     }
     return this;
   }
-  
-
-  /** Creates a sort by terms in the given field sorted
-   * according to the given locale.
-   * @param field  Name of field to sort by, cannot be <code>null</code>.
-   * @param locale Locale of values in the field.
-   */
-  public SortField (String field, Locale locale) {
-    initFieldType(field, STRING);
-    this.locale = locale;
-  }
-
-  /** Creates a sort, possibly in reverse, by terms in the given field sorted
-   * according to the given locale.
-   * @param field  Name of field to sort by, cannot be <code>null</code>.
-   * @param locale Locale of values in the field.
-   */
-  public SortField (String field, Locale locale, boolean reverse) {
-    initFieldType(field, STRING);
-    this.locale = locale;
-    this.reverse = reverse;
-  }
 
   /** Creates a sort with a custom comparison function.
    * @param field Name of field to sort by; cannot be <code>null</code>.
@@ -295,14 +271,6 @@ public class SortField {
     return type;
   }
 
-  /** Returns the Locale by which term values are interpreted.
-   * May return <code>null</code> if no Locale was specified.
-   * @return Locale, or <code>null</code>.
-   */
-  public Locale getLocale() {
-    return locale;
-  }
-
   /** Returns the instance of a {@link FieldCache} parser that fits to the given sort type.
    * May return <code>null</code> if no parser was specified. Sorting is using the default parser then.
    * @return An instance of a {@link FieldCache} parser, or <code>null</code>.
@@ -384,7 +352,6 @@ public class SortField {
         break;
     }
 
-    if (locale != null) buffer.append('(').append(locale).append(')');
     if (creator != null) buffer.append('(').append(creator).append(')');
     if (reverse) buffer.append('!');
 
@@ -404,7 +371,6 @@ public class SortField {
       other.field == this.field // field is always interned
       && other.type == this.type
       && other.reverse == this.reverse
-      && (other.locale == null ? this.locale == null : other.locale.equals(this.locale))
       && (other.comparatorSource == null ? this.comparatorSource == null : other.comparatorSource.equals(this.comparatorSource))
       && (other.creator == null ? this.creator == null : other.creator.equals(this.creator))
     );
@@ -419,7 +385,6 @@ public class SortField {
   public int hashCode() {
     int hash=type^0x346565dd + Boolean.valueOf(reverse).hashCode()^0xaf5998bb;
     if (field != null) hash += field.hashCode()^0xff5685dd;
-    if (locale != null) hash += locale.hashCode()^0x08150815;
     if (comparatorSource != null) hash += comparatorSource.hashCode();
     if (creator != null) hash += creator.hashCode()^0x3aaf56ff;
     return hash;
@@ -439,13 +404,6 @@ public class SortField {
    */
   public FieldComparator getComparator(final int numHits, final int sortPos) throws IOException {
 
-    if (locale != null) {
-      // TODO: it'd be nice to allow FieldCache.getStringIndex
-      // to optionally accept a Locale so sorting could then use
-      // the faster StringComparator impls
-      return new FieldComparator.StringComparatorLocale(numHits, field, locale);
-    }
-
     switch (type) {
     case SortField.SCORE:
       return new FieldComparator.RelevanceComparator(numHits);

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java Tue Mar 15 21:35:17 2011
@@ -1,5 +1,7 @@
 package org.apache.lucene.search;
 
+import org.apache.lucene.util.BytesRef;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -17,15 +19,13 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.text.Collator;
-
 /**
  * A Filter that restricts search results to a range of term
  * values in a given field.
  *
  * <p>This filter matches the documents looking for terms that fall into the
  * supplied range according to {@link
- * String#compareTo(String)}, unless a <code>Collator</code> is provided. It is not intended
+ * Byte#compareTo(Byte)},  It is not intended
  * for numerical ranges; use {@link NumericRangeFilter} instead.
  *
  * <p>If you construct a large number of range filters with different ranges but on the 
@@ -44,39 +44,25 @@ public class TermRangeFilter extends Mul
    *  lowerTerm is null and includeLower is true (similar for upperTerm
    *  and includeUpper)
    */
-  public TermRangeFilter(String fieldName, String lowerTerm, String upperTerm,
+  public TermRangeFilter(String fieldName, BytesRef lowerTerm, BytesRef upperTerm,
                      boolean includeLower, boolean includeUpper) {
       super(new TermRangeQuery(fieldName, lowerTerm, upperTerm, includeLower, includeUpper));
   }
 
   /**
-   * <strong>WARNING:</strong> Using this constructor and supplying a non-null
-   * value in the <code>collator</code> parameter will cause every single 
-   * index Term in the Field referenced by lowerTerm and/or upperTerm to be
-   * examined.  Depending on the number of index Terms in this Field, the 
-   * operation could be very slow.
-   *
-   * @param lowerTerm The lower bound on this range
-   * @param upperTerm The upper bound on this range
-   * @param includeLower Does this range include the lower bound?
-   * @param includeUpper Does this range include the upper bound?
-   * @param collator The collator to use when determining range inclusion; set
-   *  to null to use Unicode code point ordering instead of collation.
-   * @throws IllegalArgumentException if both terms are null or if
-   *  lowerTerm is null and includeLower is true (similar for upperTerm
-   *  and includeUpper)
+   * Factory that creates a new TermRangeFilter using Strings for term text.
    */
-  public TermRangeFilter(String fieldName, String lowerTerm, String upperTerm,
-                     boolean includeLower, boolean includeUpper,
-                     Collator collator) {
-      super(new TermRangeQuery(fieldName, lowerTerm, upperTerm, includeLower, includeUpper, collator));
+  public static TermRangeFilter newStringRange(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
+    BytesRef lower = lowerTerm == null ? null : new BytesRef(lowerTerm);
+    BytesRef upper = upperTerm == null ? null : new BytesRef(upperTerm);
+    return new TermRangeFilter(field, lower, upper, includeLower, includeUpper);
   }
-
+  
   /**
    * Constructs a filter for field <code>fieldName</code> matching
    * less than or equal to <code>upperTerm</code>.
    */
-  public static TermRangeFilter Less(String fieldName, String upperTerm) {
+  public static TermRangeFilter Less(String fieldName, BytesRef upperTerm) {
       return new TermRangeFilter(fieldName, null, upperTerm, false, true);
   }
 
@@ -84,22 +70,19 @@ public class TermRangeFilter extends Mul
    * Constructs a filter for field <code>fieldName</code> matching
    * greater than or equal to <code>lowerTerm</code>.
    */
-  public static TermRangeFilter More(String fieldName, String lowerTerm) {
+  public static TermRangeFilter More(String fieldName, BytesRef lowerTerm) {
       return new TermRangeFilter(fieldName, lowerTerm, null, true, false);
   }
   
   /** Returns the lower value of this range filter */
-  public String getLowerTerm() { return query.getLowerTerm(); }
+  public BytesRef getLowerTerm() { return query.getLowerTerm(); }
 
   /** Returns the upper value of this range filter */
-  public String getUpperTerm() { return query.getUpperTerm(); }
+  public BytesRef getUpperTerm() { return query.getUpperTerm(); }
   
   /** Returns <code>true</code> if the lower endpoint is inclusive */
   public boolean includesLower() { return query.includesLower(); }
   
   /** Returns <code>true</code> if the upper endpoint is inclusive */
   public boolean includesUpper() { return query.includesUpper(); }
-
-  /** Returns the collator used to determine range inclusion, if any. */
-  public Collator getCollator() { return query.getCollator(); }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Tue Mar 15 21:35:17 2011
@@ -18,11 +18,11 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.text.Collator;
 
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -30,7 +30,7 @@ import org.apache.lucene.util.ToStringUt
  *
  * <p>This query matches the documents looking for terms that fall into the
  * supplied range according to {@link
- * String#compareTo(String)}, unless a <code>Collator</code> is provided. It is not intended
+ * Byte#compareTo(Byte)}. It is not intended
  * for numerical ranges; use {@link NumericRangeQuery} instead.
  *
  * <p>This query uses the {@link
@@ -40,9 +40,8 @@ import org.apache.lucene.util.ToStringUt
  */
 
 public class TermRangeQuery extends MultiTermQuery {
-  private String lowerTerm;
-  private String upperTerm;
-  private Collator collator;
+  private BytesRef lowerTerm;
+  private BytesRef upperTerm;
   private boolean includeLower;
   private boolean includeUpper;
 
@@ -69,78 +68,48 @@ public class TermRangeQuery extends Mult
    *          If true, the <code>upperTerm</code> is
    *          included in the range.
    */
-  public TermRangeQuery(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
-    this(field, lowerTerm, upperTerm, includeLower, includeUpper, null);
-  }
-
-  /** Constructs a query selecting all terms greater/equal than
-   * <code>lowerTerm</code> but less/equal than <code>upperTerm</code>.
-   * <p>
-   * If an endpoint is null, it is said 
-   * to be "open". Either or both endpoints may be open.  Open endpoints may not 
-   * be exclusive (you can't select all but the first or last term without 
-   * explicitly specifying the term to exclude.)
-   * <p>
-   * If <code>collator</code> is not null, it will be used to decide whether
-   * index terms are within the given range, rather than using the Unicode code
-   * point order in which index terms are stored.
-   * <p>
-   * <strong>WARNING:</strong> Using this constructor and supplying a non-null
-   * value in the <code>collator</code> parameter will cause every single 
-   * index Term in the Field referenced by lowerTerm and/or upperTerm to be
-   * examined.  Depending on the number of index Terms in this Field, the 
-   * operation could be very slow.
-   *
-   * @param lowerTerm The Term text at the lower end of the range
-   * @param upperTerm The Term text at the upper end of the range
-   * @param includeLower
-   *          If true, the <code>lowerTerm</code> is
-   *          included in the range.
-   * @param includeUpper
-   *          If true, the <code>upperTerm</code> is
-   *          included in the range.
-   * @param collator The collator to use to collate index Terms, to determine
-   *  their membership in the range bounded by <code>lowerTerm</code> and
-   *  <code>upperTerm</code>.
-   */
-  public TermRangeQuery(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper,
-                    Collator collator) {
+  public TermRangeQuery(String field, BytesRef lowerTerm, BytesRef upperTerm, boolean includeLower, boolean includeUpper) {
     super(field);
     this.lowerTerm = lowerTerm;
     this.upperTerm = upperTerm;
     this.includeLower = includeLower;
     this.includeUpper = includeUpper;
-    this.collator = collator;
+  }
+
+  /**
+   * Factory that creates a new TermRangeQuery using Strings for term text.
+   */
+  public static TermRangeQuery newStringRange(String field, String lowerTerm, String upperTerm, boolean includeLower, boolean includeUpper) {
+    BytesRef lower = lowerTerm == null ? null : new BytesRef(lowerTerm);
+    BytesRef upper = upperTerm == null ? null : new BytesRef(upperTerm);
+    return new TermRangeQuery(field, lower, upper, includeLower, includeUpper);
   }
 
   /** Returns the lower value of this range query */
-  public String getLowerTerm() { return lowerTerm; }
+  public BytesRef getLowerTerm() { return lowerTerm; }
 
   /** Returns the upper value of this range query */
-  public String getUpperTerm() { return upperTerm; }
+  public BytesRef getUpperTerm() { return upperTerm; }
   
   /** Returns <code>true</code> if the lower endpoint is inclusive */
   public boolean includesLower() { return includeLower; }
   
   /** Returns <code>true</code> if the upper endpoint is inclusive */
   public boolean includesUpper() { return includeUpper; }
-
-  /** Returns the collator used to determine range inclusion, if any. */
-  public Collator getCollator() { return collator; }
   
   @Override
   protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
-    if (collator == null && lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) {
+    if (lowerTerm != null && upperTerm != null && lowerTerm.compareTo(upperTerm) > 0) {
       return TermsEnum.EMPTY;
     }
     
     TermsEnum tenum = terms.iterator();
     
-    if ((lowerTerm == null || (collator == null && includeLower && "".equals(lowerTerm))) && upperTerm == null) {
+    if ((lowerTerm == null || (includeLower && lowerTerm.length == 0)) && upperTerm == null) {
       return tenum;
     }
     return new TermRangeTermsEnum(tenum,
-        lowerTerm, upperTerm, includeLower, includeUpper, collator);
+        lowerTerm, upperTerm, includeLower, includeUpper);
   }
 
   /** Prints a user-readable version of this query. */
@@ -152,9 +121,10 @@ public class TermRangeQuery extends Mult
           buffer.append(":");
       }
       buffer.append(includeLower ? '[' : '{');
-      buffer.append(lowerTerm != null ? ("*".equals(lowerTerm) ? "\\*" : lowerTerm)  : "*");
+      // TODO: all these toStrings for queries should just output the bytes, it might not be UTF-8!
+      buffer.append(lowerTerm != null ? ("*".equals(lowerTerm.utf8ToString()) ? "\\*" : lowerTerm.utf8ToString())  : "*");
       buffer.append(" TO ");
-      buffer.append(upperTerm != null ? ("*".equals(upperTerm) ? "\\*" : upperTerm) : "*");
+      buffer.append(upperTerm != null ? ("*".equals(upperTerm.utf8ToString()) ? "\\*" : upperTerm.utf8ToString()) : "*");
       buffer.append(includeUpper ? ']' : '}');
       buffer.append(ToStringUtils.boost(getBoost()));
       return buffer.toString();
@@ -164,7 +134,6 @@ public class TermRangeQuery extends Mult
   public int hashCode() {
     final int prime = 31;
     int result = super.hashCode();
-    result = prime * result + ((collator == null) ? 0 : collator.hashCode());
     result = prime * result + (includeLower ? 1231 : 1237);
     result = prime * result + (includeUpper ? 1231 : 1237);
     result = prime * result + ((lowerTerm == null) ? 0 : lowerTerm.hashCode());
@@ -181,11 +150,6 @@ public class TermRangeQuery extends Mult
     if (getClass() != obj.getClass())
       return false;
     TermRangeQuery other = (TermRangeQuery) obj;
-    if (collator == null) {
-      if (other.collator != null)
-        return false;
-    } else if (!collator.equals(other.collator))
-      return false;
     if (includeLower != other.includeLower)
       return false;
     if (includeUpper != other.includeUpper)

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java?rev=1081952&r1=1081951&r2=1081952&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java Tue Mar 15 21:35:17 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.text.Collator;
 import java.util.Comparator;
 
 import org.apache.lucene.index.TermsEnum;
@@ -33,11 +32,8 @@ import org.apache.lucene.util.BytesRef;
  */
 public class TermRangeTermsEnum extends FilteredTermsEnum {
 
-  private Collator collator;
-  private String upperTermText;
-  private String lowerTermText;
-  private boolean includeLower;
-  private boolean includeUpper;
+  final private boolean includeLower;
+  final private boolean includeUpper;
   final private BytesRef lowerBytesRef;
   final private BytesRef upperBytesRef;
   private final Comparator<BytesRef> termComp;
@@ -53,79 +49,61 @@ public class TermRangeTermsEnum extends 
    * 
    * @param tenum
    *          TermsEnum to filter
-   * @param lowerTermText
+   * @param lowerTerm
    *          The term text at the lower end of the range
-   * @param upperTermText
+   * @param upperTerm
    *          The term text at the upper end of the range
    * @param includeLower
    *          If true, the <code>lowerTerm</code> is included in the range.
    * @param includeUpper
    *          If true, the <code>upperTerm</code> is included in the range.
-   * @param collator
-   *          The collator to use to collate index Terms, to determine their
-   *          membership in the range bounded by <code>lowerTerm</code> and
-   *          <code>upperTerm</code>.
    * 
    * @throws IOException
    */
-  public TermRangeTermsEnum(TermsEnum tenum, String lowerTermText, String upperTermText, 
-    boolean includeLower, boolean includeUpper, Collator collator) throws IOException {
+  public TermRangeTermsEnum(TermsEnum tenum, BytesRef lowerTerm, BytesRef upperTerm, 
+    boolean includeLower, boolean includeUpper) throws IOException {
     super(tenum);
-    this.collator = collator;
-    this.upperTermText = upperTermText;
-    this.lowerTermText = lowerTermText;
-    this.includeLower = includeLower;
-    this.includeUpper = includeUpper;
 
     // do a little bit of normalization...
     // open ended range queries should always be inclusive.
-    if (this.lowerTermText == null) {
-      this.lowerTermText = "";
+    if (lowerTerm == null) {
+      this.lowerBytesRef = new BytesRef();
       this.includeLower = true;
+    } else {
+      this.lowerBytesRef = lowerTerm;
+      this.includeLower = includeLower;
     }
-    lowerBytesRef = new BytesRef(this.lowerTermText);
 
-    if (this.upperTermText == null) {
+    if (upperTerm == null) {
       this.includeUpper = true;
       upperBytesRef = null;
     } else {
-      upperBytesRef = new BytesRef(upperTermText);
+      this.includeUpper = includeUpper;
+      upperBytesRef = upperTerm;
     }
 
-    BytesRef startBytesRef = (collator == null) ? lowerBytesRef : new BytesRef("");
-    setInitialSeekTerm(startBytesRef);
+    setInitialSeekTerm(lowerBytesRef);
     termComp = getComparator();
   }
 
   @Override
   protected AcceptStatus accept(BytesRef term) {
-    if (collator == null) {
-      if (!this.includeLower && term.equals(lowerBytesRef))
-        return AcceptStatus.NO;
-      // Use this field's default sort ordering
-      if (upperBytesRef != null) {
-        final int cmp = termComp.compare(upperBytesRef, term);
-        /*
-         * if beyond the upper term, or is exclusive and this is equal to
-         * the upper term, break out
-         */
-        if ((cmp < 0) ||
-            (!includeUpper && cmp==0)) {
-          return AcceptStatus.END;
-        }
-      }
-      return AcceptStatus.YES;
-    } else {
-      if ((includeLower
-           ? collator.compare(term.utf8ToString(), lowerTermText) >= 0
-           : collator.compare(term.utf8ToString(), lowerTermText) > 0)
-          && (upperTermText == null
-              || (includeUpper
-                  ? collator.compare(term.utf8ToString(), upperTermText) <= 0
-                  : collator.compare(term.utf8ToString(), upperTermText) < 0))) {
-        return AcceptStatus.YES;
-      }
+    if (!this.includeLower && term.equals(lowerBytesRef))
       return AcceptStatus.NO;
+    
+    // Use this field's default sort ordering
+    if (upperBytesRef != null) {
+      final int cmp = termComp.compare(upperBytesRef, term);
+      /*
+       * if beyond the upper term, or is exclusive and this is equal to
+       * the upper term, break out
+       */
+      if ((cmp < 0) ||
+          (!includeUpper && cmp==0)) {
+        return AcceptStatus.END;
+      }
     }
+
+    return AcceptStatus.YES;
   }
 }