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

svn commit: r1086876 [8/18] - in /lucene/dev/branches/realtime_search: ./ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/solr/ dev-tools/idea/solr/contrib/analysis-extras/ dev-tools/idea/solr/co...

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldComparator.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/FieldValueHitQueue.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/HitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/HitQueue.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/HitQueue.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/HitQueue.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Wed Mar 30 09:17:25 2011
@@ -72,7 +72,7 @@ public class IndexSearcher {
   private final ExecutorService executor;
 
   // the default SimilarityProvider
-  private static final SimilarityProvider defaultProvider = new DefaultSimilarity();
+  private static final SimilarityProvider defaultProvider = new DefaultSimilarityProvider();
   
   /**
    * Expert: returns a default SimilarityProvider instance.
@@ -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());
     }
   }
   
@@ -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/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/MultiPhraseQuery.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQueue.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQueue.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/PhraseQueue.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/QueryTermVector.java Wed Mar 30 09:17:25 2011
@@ -66,10 +66,10 @@ public class QueryTermVector implements 
           final TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
 
           hasMoreTokens = stream.incrementToken();
+          BytesRef bytes = termAtt.getBytesRef();
           while (hasMoreTokens) {
-            BytesRef bytes = new BytesRef();
-            termAtt.toBytesRef(bytes);
-            terms.add(bytes);
+            termAtt.fillBytesRef();
+            terms.add(new BytesRef(bytes));
             hasMoreTokens = stream.incrementToken();
           }
           processTerms(terms.toArray(new BytesRef[terms.size()]));

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

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/SortField.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeFilter.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Wed Mar 30 09:17:25 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/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/TermRangeTermsEnum.java Wed Mar 30 09:17:25 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;
   }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/package.html?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/package.html (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/function/package.html Wed Mar 30 09:17:25 2011
@@ -130,14 +130,14 @@
     Using field (byte) values to as scores:
     <p>
     Indexing:
-    <pre>
+    <pre class="prettyprint">
       f = new Field("score", "7", Field.Store.NO, Field.Index.UN_TOKENIZED);
       f.setOmitNorms(true);
       d1.add(f);
     </pre>
     <p>
     Search:
-    <pre>
+    <pre class="prettyprint">
       Query q = new FieldScoreQuery("score", FieldScoreQuery.Type.BYTE);
     </pre>
     Document d1 above would get a score of 7.
@@ -148,7 +148,7 @@
     <p>
     Dividing the original score of each document by a square root of its docid
     (just to demonstrate what it takes to manipulate scores this way)
-    <pre>
+    <pre class="prettyprint">
       Query q = queryParser.parse("my query text");
       CustomScoreQuery customQ = new CustomScoreQuery(q) {
         public float customScore(int doc, float subQueryScore, float valSrcScore) {
@@ -158,7 +158,7 @@
     </pre>
         <p>
         For more informative debug info on the custom query, also override the name() method:
-        <pre>
+        <pre class="prettyprint">
       CustomScoreQuery customQ = new CustomScoreQuery(q) {
         public float customScore(int doc, float subQueryScore, float valSrcScore) {
           return subQueryScore / Math.sqrt(docid);
@@ -171,7 +171,7 @@
         <p>
         Taking the square root of the original score and multiplying it by a "short field driven score", ie, the
         short value that was indexed for the scored doc in a certain field:
-        <pre>
+        <pre class="prettyprint">
       Query q = queryParser.parse("my query text");
       FieldScoreQuery qf = new FieldScoreQuery("shortScore", FieldScoreQuery.Type.SHORT);
       CustomScoreQuery customQ = new CustomScoreQuery(q,qf) {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java Wed Mar 30 09:17:25 2011
@@ -53,7 +53,7 @@ public class NearSpansUnordered extends 
 
   private class CellQueue extends PriorityQueue<SpansCell> {
     public CellQueue(int size) {
-      initialize(size);
+      super(size);
     }
     
     @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java Wed Mar 30 09:17:25 2011
@@ -145,7 +145,7 @@ public class SpanOrQuery extends SpanQue
 
   private class SpanQueue extends PriorityQueue<Spans> {
     public SpanQueue(int size) {
-      initialize(size);
+      super(size);
     }
 
     @Override

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/package.html?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/package.html (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/search/spans/package.html Wed Mar 30 09:17:25 2011
@@ -59,7 +59,7 @@ two starts and ends at the greater of th
 <p>For example, a span query which matches "John Kerry" within ten
 words of "George Bush" within the first 100 words of the document
 could be constructed with:
-<pre>
+<pre class="prettyprint">
 SpanQuery john   = new SpanTermQuery(new Term("content", "john"));
 SpanQuery kerry  = new SpanTermQuery(new Term("content", "kerry"));
 SpanQuery george = new SpanTermQuery(new Term("content", "george"));
@@ -82,7 +82,7 @@ SpanQuery johnKerryNearGeorgeBushAtStart
 So, for example, the above query can be restricted to documents which
 also use the word "iraq" with:
 
-<pre>
+<pre class="prettyprint">
 Query query = new BooleanQuery();
 query.add(johnKerryNearGeorgeBushAtStart, true, false);
 query.add(new TermQuery("content", "iraq"), true, false);

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

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

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/AttributeSource.java Wed Mar 30 09:17:25 2011
@@ -131,14 +131,14 @@ public class AttributeSource {
   /**
    * returns the used AttributeFactory.
    */
-  public AttributeFactory getAttributeFactory() {
+  public final AttributeFactory getAttributeFactory() {
     return this.factory;
   }
   
   /** Returns a new iterator that iterates the attribute classes
    * in the same order they were added in.
    */
-  public Iterator<Class<? extends Attribute>> getAttributeClassesIterator() {
+  public final Iterator<Class<? extends Attribute>> getAttributeClassesIterator() {
     return Collections.unmodifiableSet(attributes.keySet()).iterator();
   }
   
@@ -146,7 +146,7 @@ public class AttributeSource {
    * This iterator may contain less entries that {@link #getAttributeClassesIterator},
    * if one instance implements more than one Attribute interface.
    */
-  public Iterator<AttributeImpl> getAttributeImplsIterator() {
+  public final Iterator<AttributeImpl> getAttributeImplsIterator() {
     if (hasAttributes()) {
       if (currentState == null) {
         computeCurrentState();

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/BytesRefHash.java Wed Mar 30 09:17:25 2011
@@ -117,7 +117,7 @@ public final class BytesRefHash {
    */
   public BytesRef get(int ord, BytesRef ref) {
     assert bytesStart != null : "bytesStart is null - not initialized";
-    assert ord < bytesStart.length: "ord exceeeds byteStart len: " + bytesStart.length;
+    assert ord < bytesStart.length: "ord exceeds byteStart len: " + bytesStart.length;
     return pool.setBytesRef(ref, bytesStart[ord]);
   }
 

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

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java Wed Mar 30 09:17:25 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.util;
 
 import java.nio.CharBuffer;
 import java.nio.ByteBuffer;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadoc
 
 /**
  * Provides support for converting byte sequences to Strings and back again.
@@ -39,7 +40,10 @@ import java.nio.ByteBuffer;
  * <p/>
  *
  * @lucene.experimental
+ * @deprecated Implement {@link TermToBytesRefAttribute} and store bytes directly
+ * instead. This class will be removed in Lucene 5.0
  */
+@Deprecated
 public final class IndexableBinaryStringTools {
 
   private static final CodingCase[] CODING_CASES = {

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/PriorityQueue.java Wed Mar 30 09:17:25 2011
@@ -28,8 +28,52 @@ package org.apache.lucene.util;
 */
 public abstract class PriorityQueue<T> {
   private int size;
-  private int maxSize;
-  protected T[] heap;
+  private final int maxSize;
+  private final T[] heap;
+
+  public PriorityQueue(int maxSize) {
+    this(maxSize, true);
+  }
+
+  @SuppressWarnings("unchecked")
+  public PriorityQueue(int maxSize, boolean prepopulate) {
+    size = 0;
+    int heapSize;
+    if (0 == maxSize)
+      // We allocate 1 extra to avoid if statement in top()
+      heapSize = 2;
+    else {
+      if (maxSize == Integer.MAX_VALUE) {
+        // Don't wrap heapSize to -1, in this case, which
+        // causes a confusing NegativeArraySizeException.
+        // Note that very likely this will simply then hit
+        // an OOME, but at least that's more indicative to
+        // caller that this values is too big.  We don't +1
+        // in this case, but it's very unlikely in practice
+        // one will actually insert this many objects into
+        // the PQ:
+        heapSize = Integer.MAX_VALUE;
+      } else {
+        // NOTE: we add +1 because all access to heap is
+        // 1-based not 0-based.  heap[0] is unused.
+        heapSize = maxSize + 1;
+      }
+    }
+    heap = (T[]) new Object[heapSize]; // T is unbounded type, so this unchecked cast works always
+    this.maxSize = maxSize;
+    
+    if (prepopulate) {
+      // If sentinel objects are supported, populate the queue with them
+      T sentinel = getSentinelObject();
+      if (sentinel != null) {
+        heap[1] = sentinel;
+        for (int i = 2; i < heap.length; i++) {
+          heap[i] = getSentinelObject();
+        }
+        size = maxSize;
+      }
+    }
+  }
 
   /** Determines the ordering of objects in this priority queue.  Subclasses
    *  must define this one method.
@@ -80,45 +124,6 @@ public abstract class PriorityQueue<T> {
     return null;
   }
 
-  /** Subclass constructors must call this. */
-  @SuppressWarnings("unchecked")
-  protected final void initialize(int maxSize) {
-    size = 0;
-    int heapSize;
-    if (0 == maxSize)
-      // We allocate 1 extra to avoid if statement in top()
-      heapSize = 2;
-    else {
-      if (maxSize == Integer.MAX_VALUE) {
-        // Don't wrap heapSize to -1, in this case, which
-        // causes a confusing NegativeArraySizeException.
-        // Note that very likely this will simply then hit
-        // an OOME, but at least that's more indicative to
-        // caller that this values is too big.  We don't +1
-        // in this case, but it's very unlikely in practice
-        // one will actually insert this many objects into
-        // the PQ:
-        heapSize = Integer.MAX_VALUE;
-      } else {
-        // NOTE: we add +1 because all access to heap is
-        // 1-based not 0-based.  heap[0] is unused.
-        heapSize = maxSize + 1;
-      }
-    }
-    heap = (T[]) new Object[heapSize]; // T is unbounded type, so this unchecked cast works always
-    this.maxSize = maxSize;
-    
-    // If sentinel objects are supported, populate the queue with them
-    T sentinel = getSentinelObject();
-    if (sentinel != null) {
-      heap[1] = sentinel;
-      for (int i = 2; i < heap.length; i++) {
-        heap[i] = getSentinelObject();
-      }
-      size = maxSize;
-    }
-  }
-
   /**
    * Adds an Object to a PriorityQueue in log(size) time. If one tries to add
    * more objects than maxSize from initialize an
@@ -247,4 +252,11 @@ public abstract class PriorityQueue<T> {
     }
     heap[i] = node;				  // install saved node
   }
+  
+  /** This method returns the internal heap array as Object[].
+   * @lucene.internal
+   */
+  protected final Object[] getHeapArray() {
+    return (Object[]) heap;
+  }
 }

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/Version.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/Version.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/Version.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/Version.java Wed Mar 30 09:17:25 2011
@@ -41,6 +41,13 @@ public enum Version {
    */
   @Deprecated
   LUCENE_31,
+  
+  /**
+   * Match settings and bugs in Lucene's 3.2 release.
+   * @deprecated (4.0) Use latest
+   */
+  @Deprecated
+  LUCENE_32,
 
   /** Match settings and bugs in Lucene's 4.0 release. 
    *  <p>

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

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/Builder.java Wed Mar 30 09:17:25 2011
@@ -91,7 +91,7 @@ public class Builder<T> {
     return fst.nodeCount;
   }
 
-  public int getTermCount() {
+  public long getTermCount() {
     return frontier[0].inputCount;
   }
 
@@ -422,7 +422,7 @@ public class Builder<T> {
     Arc<T>[] arcs;
     T output;
     boolean isFinal;
-    int inputCount;
+    long inputCount;
 
     /** This node's depth, starting from the automaton root. */
     final int depth;

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FST.java Wed Mar 30 09:17:25 2011
@@ -99,7 +99,7 @@ public class FST<T> {
   public int arcWithOutputCount;
 
   // If arc has this label then that arc is final/accepted
-  public static int END_LABEL = -1;
+  public static final int END_LABEL = -1;
 
   public final static class Arc<T> {
     public int label;
@@ -297,9 +297,7 @@ public class FST<T> {
     final int v;
     if (inputType == INPUT_TYPE.BYTE1) {
       v = in.readByte()&0xFF;
-    } else if (inputType == INPUT_TYPE.BYTE2) {
-      v = in.readVInt();
-    } else {
+    } else { 
       v = in.readVInt();
     }
     return v;
@@ -478,6 +476,59 @@ public class FST<T> {
     return arc;
   }
 
+  /** Follows the <code>follow</code> arc and reads the last
+   *  arc of its target; this changes the provided
+   *  <code>arc</code> (2nd arg) in-place and returns it.
+   * 
+   * @return Returns the second argument
+   * (<code>arc</code>). */
+  public Arc<T> readLastTargetArc(Arc<T> follow, Arc<T> arc) throws IOException {
+    //System.out.println("readLast");
+    if (!targetHasArcs(follow)) {
+      //System.out.println("  end node");
+      assert follow.isFinal();
+      arc.label = -1;
+      arc.output = follow.nextFinalOutput;
+      arc.flags = BIT_LAST_ARC;
+      return arc;
+    } else {
+      final BytesReader in = getBytesReader(follow.target);
+      arc.flags = in.readByte();
+      if (arc.flag(BIT_ARCS_AS_FIXED_ARRAY)) {
+        // array: jump straight to end
+        arc.numArcs = in.readVInt();
+        arc.bytesPerArc = in.readByte() & 0xFF;
+        //System.out.println("  array numArcs=" + arc.numArcs + " bpa=" + arc.bytesPerArc);
+        arc.posArcsStart = in.pos;
+        arc.arcIdx = arc.numArcs - 2;
+      } else {
+        // non-array: linear scan
+        arc.bytesPerArc = 0;
+        //System.out.println("  scan");
+        while(!arc.isLast()) {
+          // skip this arc:
+          readLabel(in);
+          if (arc.flag(BIT_ARC_HAS_OUTPUT)) {
+            outputs.read(in);
+          }
+          if (arc.flag(BIT_ARC_HAS_FINAL_OUTPUT)) {
+            outputs.read(in);
+          }
+          if (arc.flag(BIT_STOP_NODE)) {
+          } else if (arc.flag(BIT_TARGET_NEXT)) {
+          } else {
+            in.pos -= 4;
+          }
+          arc.flags = in.readByte();
+        }
+        arc.nextArc = in.pos+1;
+      }
+      readNextRealArc(arc);
+      assert arc.isLast();
+      return arc;
+    }
+  }
+
   /**
    * Follow the <code>follow</code> arc and read the first arc of its target;
    * this changes the provided <code>arc</code> (2nd arg) in-place and returns
@@ -518,15 +569,13 @@ public class FST<T> {
       arc.numArcs = in.readVInt();
       arc.bytesPerArc = in.readByte() & 0xFF;
       arc.arcIdx = -1;
-      arc.posArcsStart = in.pos;
+      arc.nextArc = arc.posArcsStart = in.pos;
       //System.out.println("  bytesPer=" + arc.bytesPerArc + " numArcs=" + arc.numArcs + " arcsStart=" + pos);
     } else {
-      in.pos++;
+      arc.nextArc = address;
       arc.bytesPerArc = 0;
     }
-    arc.nextArc = in.pos;
-    arc.label = 0;
-    return readNextArc(arc);
+    return readNextRealArc(arc);
   }
 
   /**
@@ -598,6 +647,7 @@ public class FST<T> {
     if (arc.bytesPerArc != 0) {
       // arcs are at fixed entries
       arc.arcIdx++;
+      assert arc.arcIdx < arc.numArcs;
       in = getBytesReader(arc.posArcsStart - arc.arcIdx*arc.bytesPerArc);
     } else {
       // arcs are packed

Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/util/automaton/fst/FSTEnum.java Wed Mar 30 09:17:25 2011
@@ -274,7 +274,7 @@ abstract class FSTEnum<T> {
     while(true) {
       //System.out.println("  cycle upto=" + upto + " arc.label=" + arc.label + " (" + (char) arc.label + ") targetLabel=" + targetLabel + " isLast?=" + arc.isLast());
 
-      if (arc.bytesPerArc != 0 && arc.label != -1) {
+      if (arc.bytesPerArc != 0 && arc.label != FST.END_LABEL) {
         // Arcs are fixed array -- use binary search to find
         // the target.
 
@@ -465,12 +465,7 @@ abstract class FSTEnum<T> {
       }
       incr();
 
-      final FST.Arc<T> nextArc = getArc(upto);
-      fst.readFirstTargetArc(arc, nextArc);
-      arc = nextArc;
-      while(!arc.isLast()) {
-        fst.readNextArc(arc);
-      }
+      arc = fst.readLastTargetArc(arc, getArc(upto));
     }
   }
 

Modified: lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo.xml?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo.xml (original)
+++ lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo.xml Wed Mar 30 09:17:25 2011
@@ -28,23 +28,14 @@ functionalities of Lucene and how you ca
 <section id="Setting your CLASSPATH"><title>Setting your CLASSPATH</title>
 <p>
 First, you should <a href="http://www.apache.org/dyn/closer.cgi/lucene/java/">download</a> the
-latest Lucene distribution and then extract it to a working directory.  Alternatively, you can <a
-href="http://wiki.apache.org/lucene-java/SourceRepository">check out the sources from
-Subversion</a>, and then run <code>ant</code> in the <code>lucene/contrib/demo/</code>
-directory to generate the JARs.
+latest Lucene distribution and then extract it to a working directory.
 </p>
 <p>
 You need three JARs: the Lucene JAR, the common analysis JAR, and the Lucene demo JAR.  You should
 see the Lucene JAR file in the directory you created when you extracted the archive -- it
 should be named something like <code>lucene-core-{version}.jar</code>.  You should also see files
-called <code>lucene-analysis-common-{version}.jar</code> and <code>lucene-demos-{version}.jar</code>.
-If you checked out the sources from Subversion then the JARs are located at:
+called <code>lucene-analyzers-common-{version}.jar</code> and <code>lucene-demo-{version}.jar</code>.
 </p>
-<ul>
-  <li><code>lucene/build/lucene-core-4.0-SNAPSHOT.jar</code></li>
-  <li><code>modules/analysis/build/common/lucene-analyzers-common-4.0-SNAPSHOT.jar</code></li>
-  <li><code>lucene/build/contrib/demo/lucene-demo-4.0-SNAPSHOT.jar</code></li>
-</ul>
 <p>
 Put all three of these files in your Java CLASSPATH.
 </p>

Modified: lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml (original)
+++ lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/demo2.xml Wed Mar 30 09:17:25 2011
@@ -22,7 +22,12 @@ how to use Lucene in their applications.
 <section id="Location of the source"><title>Location of the source</title>
 
 <p>
-Relative to the directory created when you extracted Lucene or retrieved it from Subversion, you
+NOTE: to examine the sources, you need to download and extract a source checkout of 
+Lucene: (lucene-{version}-src.zip).
+</p>
+
+<p>
+Relative to the directory created when you extracted Lucene, you
 should see a directory called <code>lucene/contrib/demo/</code>.  This is the root for the Lucene
 demo.  Under this directory is <code>src/java/org/apache/lucene/demo/</code>.  This is where all
 the Java sources for the demo live.
@@ -76,7 +81,7 @@ first wipe the slate clean before indexi
 <p>
 Lucene <a href="api/core/org/apache/lucene/store/Directory.html">Directory</a>s are used by the
 <code>IndexWriter</code> to store information in the index.  In addition to the 
-<a href="api/core/org/apache/lucen/store/FSDirectory.html">FSDirectory</a> implementation we are using,
+<a href="api/core/org/apache/lucene/store/FSDirectory.html">FSDirectory</a> implementation we are using,
 there are several other <code>Directory</code> subclasses that can write to RAM, to databases, etc.
 </p>
   
@@ -90,8 +95,8 @@ Standard Annex #29</a>; converts tokens 
 common language words such as articles (a, an, the, etc.) and other tokens that may have less value for
 searching.  It should be noted that there are different rules for every language, and you should use the
 proper analyzer for each.  Lucene currently provides Analyzers for a number of different languages (see
-the <code>*Analyzer.java</code> sources under 
-<a href="http://svn.apache.org/repos/asf/lucene/dev/trunk/modules/analysis/common/src/java/org/apache/lucene/analysis/"
+the javadocs under 
+<a href="api/module-analysis-common/org/apache/lucene/analysis/package-summary.html"
 >modules/analysis/common/src/java/org/apache/lucene/analysis</a>).
 </p>
 

Modified: lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/fileformats.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/fileformats.xml?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/fileformats.xml (original)
+++ lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/fileformats.xml Wed Mar 30 09:17:25 2011
@@ -87,6 +87,10 @@
 	        uncompressed). See issue LUCENE-1960 for details.
             </p>
 
+        <p>
+            In version 3.1, segments records the code version
+            that created them. See LUCENE-2720 for details.
+           </p>
         </section>
 
         <section id="Definitions"><title>Definitions</title>
@@ -928,8 +932,8 @@
                     written twice.
                 </p>
                 <p>
-                    <b>2.9</b>
-                    Segments --&gt; Format, Version, NameCounter, SegCount, &lt;SegName, SegSize, DelGen, DocStoreOffset, [DocStoreSegment, DocStoreIsCompoundFile], HasSingleNormFile, NumField,
+                    <b>3.1</b>
+                    Segments --&gt; Format, Version, NameCounter, SegCount, &lt;SegVersion, SegName, SegSize, DelGen, DocStoreOffset, [DocStoreSegment, DocStoreIsCompoundFile], HasSingleNormFile, NumField,
                     NormGen<sup>NumField</sup>,
                     IsCompoundFile, DeletionCount, HasProx, Diagnostics&gt;<sup>SegCount</sup>, CommitUserData, Checksum
                 </p>
@@ -944,7 +948,7 @@
                 </p>
 
                 <p>
-                   SegName, DocStoreSegment --&gt; String
+                   SegVersion, SegName, DocStoreSegment --&gt; String
                 </p>
 
 		<p>
@@ -974,6 +978,10 @@
                 </p>
 
                 <p>
+                    SegVersion is the code version that created the segment.
+                </p>
+
+                <p>
                     SegName is the name of the segment, and is used as the file name prefix
                     for all of the files that compose the segment's index.
                 </p>

Modified: lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/site.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/site.xml?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/site.xml (original)
+++ lucene/dev/branches/realtime_search/lucene/src/site/src/documentation/content/xdocs/site.xml Wed Mar 30 09:17:25 2011
@@ -50,6 +50,7 @@ See http://forrest.apache.org/docs/linki
       <javadoc label="Javadocs">
          <javadoc-all label="All" href="ext:javadocs-all"/>
 		 <javadoc-core label="Core" href="ext:javadocs-core"/>
+		 <javadoc-test-framework label="Test Framework" href="ext:javadocs-test-framework"/>
 		 <javadoc-contrib label="Contrib">
 		    <javadoc-contrib-ant label="Ant" href="ext:javadocs-contrib-ant"/>
 		    <javadoc-contrib-bdb label="Bdb" href="ext:javadocs-contrib-bdb"/>
@@ -98,6 +99,7 @@ See http://forrest.apache.org/docs/linki
     <changes-contrib href="changes/Contrib-Changes.html" />
     <javadocs-all href="api/all/index.html"/>
 	<javadocs-core href="api/core/index.html"/>
+	<javadocs-test-framework href="api/test-framework/index.html"/>
 	<javadocs-contrib-ant href="api/contrib-ant/index.html"/>
 	<javadocs-contrib-bdb href="api/contrib-bdb/index.html"/>
 	<javadocs-contrib-bdb-je href="api/contrib-bdb-je/index.html"/>

Modified: lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Wed Mar 30 09:17:25 2011
@@ -24,6 +24,7 @@ import java.util.Random;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IndexWriter; // javadoc
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.Version;
@@ -127,9 +128,28 @@ public class RandomIndexWriter implement
   }
 
   public IndexReader getReader() throws IOException {
-    getReaderCalled = true;
-    if (r.nextInt(4) == 2)
+
+    return getReader(true);
+  }
+
+  private void doRandomOptimize() throws IOException {
+    final int segCount = w.getSegmentCount();
+    if (r.nextBoolean() || segCount == 0) {
+      // full optimize
       w.optimize();
+    } else {
+      // partial optimize
+      final int limit = _TestUtil.nextInt(r, 1, segCount);
+      w.optimize(limit);
+      assert w.getSegmentCount() <= limit: "limit=" + limit + " actual=" + w.getSegmentCount();
+    }
+  }
+
+  public IndexReader getReader(boolean applyDeletions) throws IOException {
+    getReaderCalled = true;
+    if (r.nextInt(4) == 2) {
+      doRandomOptimize();
+    }
     // If we are writing with PreFlexRW, force a full
     // IndexReader.open so terms are sorted in codepoint
     // order during searching:
@@ -151,7 +171,7 @@ public class RandomIndexWriter implement
     // if someone isn't using getReader() API, we want to be sure to
     // maybeOptimize since presumably they might open a reader on the dir.
     if (getReaderCalled == false && r.nextInt(4) == 2) {
-      w.optimize();
+      doRandomOptimize();
     }
     w.close();
   }

Modified: lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java Wed Mar 30 09:17:25 2011
@@ -26,7 +26,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.CodecUtil;
 
 /** Reads IndexInputs written with {@link
- *  SingleIntIndexOutput}.  NOTE: this class is just for
+ *  MockSingleIntIndexOutput}.  NOTE: this class is just for
  *  demonstration puprposes (it is a very slow way to read a
  *  block of ints).
  *

Modified: lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java?rev=1086876&r1=1086875&r2=1086876&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/test-framework/org/apache/lucene/search/CheckHits.java Wed Mar 30 09:17:25 2011
@@ -268,7 +268,7 @@ public class CheckHits {
    * query corresponds with the true score. 
    *
    * @see ExplanationAsserter
-   * @see #checkExplanations(Query, String, Searcher, boolean) for a
+   * @see #checkExplanations(Query, String, IndexSearcher, boolean) for a
    * "deep" testing of the explanation details.
    *   
    * @param query the query to test