You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ry...@apache.org on 2012/03/07 00:17:23 UTC

svn commit: r1297785 [4/16] - in /lucene/dev/branches/lucene3795_lsp_spatial_module: ./ dev-tools/eclipse/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/contrib/demo/ dev-tools/maven/lucene/contrib/highlighter/ dev-tools/maven/lucene/...

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FilteredDocIdSet.java Tue Mar  6 23:17:08 2012
@@ -84,7 +84,11 @@ public abstract class FilteredDocIdSet e
    */
   @Override
   public DocIdSetIterator iterator() throws IOException {
-    return new FilteredDocIdSetIterator(_innerSet.iterator()) {
+    final DocIdSetIterator iterator = _innerSet.iterator();
+    if (iterator == null) {
+      return null;
+    }
+    return new FilteredDocIdSetIterator(iterator) {
       @Override
       protected boolean match(int docid) {
         return FilteredDocIdSet.this.match(docid);

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Tue Mar  6 23:17:08 2012
@@ -287,7 +287,7 @@ public final class FuzzyTermsEnum extend
   }
   
   @Override
-  public Comparator<BytesRef> getComparator() throws IOException {
+  public Comparator<BytesRef> getComparator() {
     return actualEnum.getComparator();
   }
   

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Tue Mar  6 23:17:08 2012
@@ -408,16 +408,12 @@ class UnionDocsAndPositionsEnum extends 
       Iterator<DocsAndPositionsEnum> i = docsEnums.iterator();
       while (i.hasNext()) {
         DocsAndPositionsEnum postings = i.next();
-        if (postings.nextDoc() != DocsAndPositionsEnum.NO_MORE_DOCS) {
+        if (postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
           add(postings);
         }
       }
     }
 
-    final public DocsEnum peek() {
-      return top();
-    }
-
     @Override
     public final boolean lessThan(DocsAndPositionsEnum a, DocsAndPositionsEnum b) {
       return a.docID() < b.docID();

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Tue Mar  6 23:17:08 2012
@@ -60,6 +60,7 @@ public class MultiTermQueryWrapperFilter
   }
 
   @Override
+  @SuppressWarnings({"unchecked","rawtypes"})
   public final boolean equals(final Object o) {
     if (o==this) return true;
     if (o==null) return false;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/NumericRangeQuery.java Tue Mar  6 23:17:08 2012
@@ -352,6 +352,7 @@ public final class NumericRangeQuery<T e
   }
 
   @Override
+  @SuppressWarnings({"unchecked","rawtypes"})
   public final boolean equals(final Object o) {
     if (o==this) return true;
     if (!super.equals(o))

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/SortField.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/SortField.java Tue Mar  6 23:17:08 2012
@@ -376,7 +376,7 @@ public class SortField {
    *   optimize themselves when they are the primary sort.
    * @return {@link FieldComparator} to use when sorting
    */
-  public FieldComparator getComparator(final int numHits, final int sortPos) throws IOException {
+  public FieldComparator<?> getComparator(final int numHits, final int sortPos) throws IOException {
 
     switch (type) {
     case SCORE:

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java Tue Mar  6 23:17:08 2012
@@ -17,12 +17,12 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.ThreadInterruptedException;
 
+import java.io.IOException;
+
 /**
  * The {@link TimeLimitingCollector} is used to timeout search requests that
  * take longer than the maximum allowed search time limit. After this time is
@@ -60,7 +60,7 @@ public class TimeLimitingCollector exten
 
   private long t0 = Long.MIN_VALUE;
   private long timeout = Long.MIN_VALUE;
-  private final Collector collector;
+  private Collector collector;
   private final Counter clock;
   private final long ticksAllowed;
   private boolean greedy = false;
@@ -172,6 +172,17 @@ public class TimeLimitingCollector exten
   public boolean acceptsDocsOutOfOrder() {
     return collector.acceptsDocsOutOfOrder();
   }
+  
+  /**
+   * This is so the same timer can be used with a multi-phase search process such as grouping. 
+   * We don't want to create a new TimeLimitingCollector for each phase because that would 
+   * reset the timer for each phase.  Once time is up subsequent phases need to timeout quickly.
+   *
+   * @param collector The actual collector performing search functionality
+   */
+  public void setCollector(Collector collector) {
+    this.collector = collector;
+  }
 
 
   /**

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopDocs.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopDocs.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopDocs.java Tue Mar  6 23:17:08 2012
@@ -116,10 +116,11 @@ public class TopDocs {
     }
   }
 
+  @SuppressWarnings({"rawtypes","unchecked"})
   private static class MergeSortQueue extends PriorityQueue<ShardRef> {
     // These are really FieldDoc instances:
     final ScoreDoc[][] shardHits;
-    final FieldComparator[] comparators;
+    final FieldComparator<?>[] comparators;
     final int[] reverseMul;
 
     public MergeSortQueue(Sort sort, TopDocs[] shardHits) throws IOException {
@@ -155,7 +156,7 @@ public class TopDocs {
     }
 
     // Returns true if first is < second
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked","rawtypes"})
     public boolean lessThan(ShardRef first, ShardRef second) {
       assert first != second;
       final FieldDoc firstFD = (FieldDoc) shardHits[first.shardIndex][first.hitIndex];

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java Tue Mar  6 23:17:08 2012
@@ -46,7 +46,7 @@ public abstract class TopFieldCollector 
   private static class OneComparatorNonScoringCollector extends 
       TopFieldCollector {
 
-    FieldComparator comparator;
+    FieldComparator<?> comparator;
     final int reverseMul;
     final FieldValueHitQueue<Entry> queue;
     
@@ -70,7 +70,7 @@ public abstract class TopFieldCollector 
       if (queueFull) {
         if ((reverseMul * comparator.compareBottom(doc)) <= 0) {
           // since docs are visited in doc Id order, if compare is 0, it means
-          // this document is largest than anything else in the queue, and
+          // this document is larger than anything else in the queue, and
           // therefore not competitive.
           return;
         }
@@ -382,7 +382,7 @@ public abstract class TopFieldCollector 
    */
   private static class MultiComparatorNonScoringCollector extends TopFieldCollector {
     
-    final FieldComparator[] comparators;
+    final FieldComparator<?>[] comparators;
     final int[] reverseMul;
     final FieldValueHitQueue<Entry> queue;
     public MultiComparatorNonScoringCollector(FieldValueHitQueue<Entry> queue,

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java Tue Mar  6 23:17:08 2012
@@ -135,7 +135,8 @@ public class PayloadSpanUtil {
           }
         }
 
-        @SuppressWarnings("unchecked") final List<Query>[] disjunctLists = new List[maxPosition + 1];
+        @SuppressWarnings({"rawtypes","unchecked"}) final List<Query>[] disjunctLists =
+            new List[maxPosition + 1];
         int distinctPositions = 0;
 
         for (int i = 0; i < termArrays.size(); ++i) {

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Tue Mar  6 23:17:08 2012
@@ -25,7 +25,6 @@ import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.ComplexExplanation;
-import org.apache.lucene.search.payloads.PayloadNearQuery.PayloadNearSpanScorer;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java Tue Mar  6 23:17:08 2012
@@ -60,6 +60,7 @@ public class SpanMultiTermQueryWrapper<Q
    * Be sure to not change the rewrite method on the wrapped query afterwards! Doing so will
    * throw {@link UnsupportedOperationException} on rewriting this query!
    */
+  @SuppressWarnings({"rawtypes","unchecked"})
   public SpanMultiTermQueryWrapper(Q query) {
     this.query = query;
     
@@ -123,6 +124,7 @@ public class SpanMultiTermQueryWrapper<Q
   }
 
   @Override
+  @SuppressWarnings({"rawtypes","unchecked"})
   public boolean equals(Object obj) {
     if (this == obj) return true;
     if (obj == null) return false;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java Tue Mar  6 23:17:08 2012
@@ -56,7 +56,7 @@ public class TermSpans extends Spans {
         return false;
       }
       doc = postings.nextDoc();
-      if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) {
+      if (doc == DocIdSetIterator.NO_MORE_DOCS) {
         return false;
       }
       freq = postings.freq();
@@ -70,7 +70,7 @@ public class TermSpans extends Spans {
   @Override
   public boolean skipTo(int target) throws IOException {
     doc = postings.advance(target);
-    if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) {
+    if (doc == DocIdSetIterator.NO_MORE_DOCS) {
       return false;
     }
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ByteBlockPool.java Tue Mar  6 23:17:08 2012
@@ -281,6 +281,37 @@ public final class ByteBlockPool {
   }
   
   /**
+   *
+   */
+  public final BytesRef copyFrom(final BytesRef bytes) {
+    final int length = bytes.length;
+    final int offset = bytes.offset;
+    bytes.offset = 0;
+    bytes.grow(length);
+    int bufferIndex = offset >> BYTE_BLOCK_SHIFT;
+    byte[] buffer = buffers[bufferIndex];
+    int pos = offset & BYTE_BLOCK_MASK;
+    int overflow = (pos + length) - BYTE_BLOCK_SIZE;
+    do {
+      if (overflow <= 0) {
+        System.arraycopy(buffer, pos, bytes.bytes, bytes.offset, bytes.length);
+        bytes.length = length;
+        bytes.offset = 0;
+        break;
+      } else {
+        final int bytesToCopy = length - overflow;
+        System.arraycopy(buffer, pos, bytes.bytes, bytes.offset, bytesToCopy);
+        pos = 0;
+        bytes.length -= bytesToCopy;
+        bytes.offset += bytesToCopy;
+        buffer = buffers[++bufferIndex];
+        overflow = overflow - BYTE_BLOCK_SIZE;
+      }
+    } while (true);
+    return bytes;
+  }
+  
+  /**
    * Writes the pools content to the given {@link DataOutput}
    */
   public final void writePool(final DataOutput out) throws IOException {

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/BytesRef.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/BytesRef.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/BytesRef.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/BytesRef.java Tue Mar  6 23:17:08 2012
@@ -233,13 +233,7 @@ public final class BytesRef implements C
       final byte[] bBytes = b.bytes;
       int bUpto = b.offset;
       
-      final int aStop;
-      if (a.length < b.length) {
-        aStop = aUpto + a.length;
-      } else {
-        aStop = aUpto + b.length;
-      }
-
+      final int aStop = aUpto + Math.min(a.length, b.length);
       while(aUpto < aStop) {
         int aByte = aBytes[aUpto++] & 0xff;
         int bByte = bBytes[bUpto++] & 0xff;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/FieldCacheSanityChecker.java Tue Mar  6 23:17:08 2012
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.CompositeReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.FieldCache;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/IndexableBinaryStringTools.java Tue Mar  6 23:17:08 2012
@@ -17,8 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.nio.CharBuffer;
-import java.nio.ByteBuffer;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadoc
 
 /**

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ReaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ReaderUtil.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ReaderUtil.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/ReaderUtil.java Tue Mar  6 23:17:08 2012
@@ -17,13 +17,10 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-import java.util.ArrayList;
 import java.util.List;
 import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReaderContext;
-import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.CompositeReader;
 import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/Automaton.java Tue Mar  6 23:17:08 2012
@@ -430,7 +430,7 @@ public class Automaton implements Clonea
       }
     }
     // map<state, set<state>>
-    @SuppressWarnings("unchecked") Set<State> map[] = new Set[states.length];
+    @SuppressWarnings({"rawtypes","unchecked"}) Set<State> map[] = new Set[states.length];
     for (int i = 0; i < map.length; i++)
       map[i] = new HashSet<State>();
     for (State s : states) {

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/automaton/MinimizationOperations.java Tue Mar  6 23:17:08 2012
@@ -74,11 +74,11 @@ final public class MinimizationOperation
     final int[] sigma = a.getStartPoints();
     final State[] states = a.getNumberedStates();
     final int sigmaLen = sigma.length, statesLen = states.length;
-    @SuppressWarnings("unchecked") final ArrayList<State>[][] reverse =
+    @SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<State>[][] reverse =
       (ArrayList<State>[][]) new ArrayList[statesLen][sigmaLen];
-    @SuppressWarnings("unchecked") final HashSet<State>[] partition =
+    @SuppressWarnings({"rawtypes","unchecked"}) final HashSet<State>[] partition =
       (HashSet<State>[]) new HashSet[statesLen];
-    @SuppressWarnings("unchecked") final ArrayList<State>[] splitblock =
+    @SuppressWarnings({"rawtypes","unchecked"}) final ArrayList<State>[] splitblock =
       (ArrayList<State>[]) new ArrayList[statesLen];
     final int[] block = new int[statesLen];
     final StateList[][] active = new StateList[statesLen][sigmaLen];

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Builder.java Tue Mar  6 23:17:08 2012
@@ -144,7 +144,8 @@ public class Builder<T> {
     }
     NO_OUTPUT = outputs.getNoOutput();
 
-    @SuppressWarnings("unchecked") final UnCompiledNode<T>[] f = (UnCompiledNode<T>[]) new UnCompiledNode[10];
+    @SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T>[] f =
+        (UnCompiledNode<T>[]) new UnCompiledNode[10];
     frontier = f;
     for(int idx=0;idx<frontier.length;idx++) {
       frontier[idx] = new UnCompiledNode<T>(this, idx);
@@ -239,7 +240,8 @@ public class Builder<T> {
         if (node.inputCount < minSuffixCount2 || (minSuffixCount2 == 1 && node.inputCount == 1 && idx > 1)) {
           // drop all arcs
           for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
-            @SuppressWarnings("unchecked") final UnCompiledNode<T> target = (UnCompiledNode<T>) node.arcs[arcIdx].target;
+            @SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T> target =
+                (UnCompiledNode<T>) node.arcs[arcIdx].target;
             target.clear();
           }
           node.numArcs = 0;
@@ -356,7 +358,7 @@ public class Builder<T> {
     final int prefixLenPlus1 = pos1+1;
       
     if (frontier.length < input.length+1) {
-      @SuppressWarnings("unchecked") final UnCompiledNode<T>[] next =
+      @SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T>[] next =
         new UnCompiledNode[ArrayUtil.oversize(input.length+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       System.arraycopy(frontier, 0, next, 0, frontier.length);
       for(int idx=frontier.length;idx<next.length;idx++) {
@@ -458,7 +460,7 @@ public class Builder<T> {
       final Arc<T> arc = node.arcs[arcIdx];
       if (!arc.target.isCompiled()) {
         // not yet compiled
-        @SuppressWarnings("unchecked") final UnCompiledNode<T> n = (UnCompiledNode<T>) arc.target;
+        @SuppressWarnings({"rawtypes","unchecked"}) final UnCompiledNode<T> n = (UnCompiledNode<T>) arc.target;
         if (n.numArcs == 0) {
           //System.out.println("seg=" + segment + "        FORCE final arc=" + (char) arc.label);
           arc.isFinal = n.isFinal = true;
@@ -512,7 +514,7 @@ public class Builder<T> {
      *          LUCENE-2934 (node expansion based on conditions other than the
      *          fanout size).
      */
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"rawtypes","unchecked"})
     public UnCompiledNode(Builder<T> owner, int depth) {
       this.owner = owner;
       arcs = (Arc<T>[]) new Arc[1];
@@ -545,7 +547,7 @@ public class Builder<T> {
       assert label >= 0;
       assert numArcs == 0 || label > arcs[numArcs-1].label: "arc[-1].label=" + arcs[numArcs-1].label + " new label=" + label + " numArcs=" + numArcs;
       if (numArcs == arcs.length) {
-        @SuppressWarnings("unchecked") final Arc<T>[] newArcs =
+        @SuppressWarnings({"rawtypes","unchecked"}) final Arc<T>[] newArcs =
           new Arc[ArrayUtil.oversize(numArcs+1, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
         System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
         for(int arcIdx=numArcs;arcIdx<newArcs.length;arcIdx++) {

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Tue Mar  6 23:17:08 2012
@@ -376,7 +376,7 @@ public final class FST<T> {
   }
 
   // Caches first 128 labels
-  @SuppressWarnings("unchecked")
+  @SuppressWarnings({"rawtypes","unchecked"})
   private void cacheRootArcs() throws IOException {
     cachedRootArcs = (Arc<T>[]) new Arc[0x80];
     final Arc<T> arc = new Arc<T>();
@@ -840,6 +840,7 @@ public final class FST<T> {
   }
 
   public Arc<T> readFirstRealTargetArc(int node, Arc<T> arc, final BytesReader in) throws IOException {
+    assert in.bytes == bytes;
     final int address = getNodeAddress(node);
     in.pos = address;
     //System.out.println("  readFirstRealTargtArc address="
@@ -936,6 +937,7 @@ public final class FST<T> {
   /** Never returns null, but you should never call this if
    *  arc.isLast() is true. */
   public Arc<T> readNextRealArc(Arc<T> arc, final BytesReader in) throws IOException {
+    assert in.bytes == bytes;
 
     // TODO: can't assert this because we call from readFirstArc
     // assert !flag(arc.flags, BIT_LAST_ARC);
@@ -1019,6 +1021,7 @@ public final class FST<T> {
    *  This returns null if the arc was not found, else the incoming arc. */
   public Arc<T> findTargetArc(int labelToMatch, Arc<T> follow, Arc<T> arc, BytesReader in) throws IOException {
     assert cachedRootArcs != null;
+    assert in.bytes == bytes;
 
     if (labelToMatch == END_LABEL) {
       if (follow.isFinal()) {
@@ -1225,17 +1228,20 @@ public final class FST<T> {
 
   /** Expert */
   public static abstract class BytesReader extends DataInput {
-    int pos;
+    protected int pos;
+    protected final byte[] bytes;
+    protected BytesReader(byte[] bytes, int pos) {
+      this.bytes = bytes;
+      this.pos = pos;
+    }
     abstract void skip(int byteCount);
     abstract void skip(int base, int byteCount);
   }
 
   final static class ReverseBytesReader extends BytesReader {
-    final byte[] bytes;
 
     public ReverseBytesReader(byte[] bytes, int pos) {
-      this.bytes = bytes;
-      this.pos = pos;
+      super(bytes, pos);
     }
 
     @Override
@@ -1262,11 +1268,9 @@ public final class FST<T> {
   // TODO: can we use just ByteArrayDataInput...?  need to
   // add a .skipBytes to DataInput.. hmm and .setPosition
   final static class ForwardBytesReader extends BytesReader {
-    final byte[] bytes;
 
     public ForwardBytesReader(byte[] bytes, int pos) {
-      this.bytes = bytes;
-      this.pos = pos;
+      super(bytes, pos);
     }
 
     @Override

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/FSTEnum.java Tue Mar  6 23:17:08 2012
@@ -30,9 +30,9 @@ import java.io.IOException;
 abstract class FSTEnum<T> {
   protected final FST<T> fst;
 
-  @SuppressWarnings("unchecked") protected FST.Arc<T>[] arcs = new FST.Arc[10];
+  @SuppressWarnings({"rawtypes","unchecked"}) protected FST.Arc<T>[] arcs = new FST.Arc[10];
   // outputs are cumulative
-  @SuppressWarnings("unchecked") protected T[] output = (T[]) new Object[10];
+  @SuppressWarnings({"rawtypes","unchecked"}) protected T[] output = (T[]) new Object[10];
 
   protected final T NO_OUTPUT;
   protected final FST.Arc<T> scratchArc = new FST.Arc<T>();
@@ -462,13 +462,13 @@ abstract class FSTEnum<T> {
     upto++;
     grow();
     if (arcs.length <= upto) {
-      @SuppressWarnings("unchecked") final FST.Arc<T>[] newArcs =
+      @SuppressWarnings({"rawtypes","unchecked"}) final FST.Arc<T>[] newArcs =
         new FST.Arc[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       System.arraycopy(arcs, 0, newArcs, 0, arcs.length);
       arcs = newArcs;
     }
     if (output.length <= upto) {
-      @SuppressWarnings("unchecked") final T[] newOutput =
+      @SuppressWarnings({"rawtypes","unchecked"}) final T[] newOutput =
         (T[]) new Object[ArrayUtil.oversize(1+upto, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
       System.arraycopy(output, 0, newOutput, 0, output.length);
       output = newOutput;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Tue Mar  6 23:17:08 2012
@@ -83,11 +83,6 @@ public final class Util {
     }
   }
 
-  // TODO: parameterize the FST type <T> and allow passing in a
-  // comparator; eg maybe your output is a PairOutput and
-  // one of the outputs in the pair is monotonic so you
-  // compare by that
-
   /** Reverse lookup (lookup by output instead of by input),
    *  in the special case when your FSTs outputs are
    *  strictly ascending.  This locates the input/output
@@ -133,7 +128,7 @@ public final class Util {
         }
       }
 
-      if (fst.targetHasArcs(arc)) {
+      if (FST.targetHasArcs(arc)) {
         //System.out.println("  targetHasArcs");
         if (result.ints.length == upto) {
           result.grow(1+upto);
@@ -155,7 +150,7 @@ public final class Util {
             final byte flags = in.readByte();
             fst.readLabel(in);
             final long minArcOutput;
-            if ((flags & fst.BIT_ARC_HAS_OUTPUT) != 0) {
+            if ((flags & FST.BIT_ARC_HAS_OUTPUT) != 0) {
               final long arcOutput = fst.outputs.read(in);
               minArcOutput = output + arcOutput;
             } else {
@@ -235,14 +230,16 @@ public final class Util {
     }    
   }
 
-  private static class FSTPath implements Comparable<FSTPath> {
-    public FST.Arc<Long> arc;
-    public long cost;
+  private static class FSTPath<T> implements Comparable<FSTPath<T>> {
+    public FST.Arc<T> arc;
+    public T cost;
     public final IntsRef input = new IntsRef();
+    final Comparator<T> comparator;
 
-    public FSTPath(long cost, FST.Arc<Long> arc) {
-      this.arc = new FST.Arc<Long>().copyFrom(arc);
+    public FSTPath(T cost, FST.Arc<T> arc, Comparator<T> comparator) {
+      this.arc = new FST.Arc<T>().copyFrom(arc);
       this.cost = cost;
+      this.comparator = comparator;
     }
 
     @Override
@@ -251,48 +248,50 @@ public final class Util {
     }
 
     @Override
-    public int compareTo(FSTPath other) {
-      if (cost < other.cost) {
-        return -1;
-      } else if (cost > other.cost) {
-        return 1;
-      } else  {
+    public int compareTo(FSTPath<T> other) {
+      int cmp = comparator.compare(cost, other.cost);
+      if (cmp == 0) {
         return input.compareTo(other.input);
+      } else {
+        return cmp;
       }
     }
   }
 
-  private static class TopNSearcher {
+  private static class TopNSearcher<T> {
 
-    private final FST<Long> fst;
-    private final FST.Arc<Long> fromNode;
+    private final FST<T> fst;
+    private final FST.Arc<T> fromNode;
     private final int topN;
+    
+    final Comparator<T> comparator;
 
     // Set once the queue has filled:
-    FSTPath bottom = null;
+    FSTPath<T> bottom = null;
 
-    TreeSet<FSTPath> queue = null;
+    TreeSet<FSTPath<T>> queue = null;
 
-    public TopNSearcher(FST<Long> fst, FST.Arc<Long> fromNode, int topN) {
+    public TopNSearcher(FST<T> fst, FST.Arc<T> fromNode, int topN, Comparator<T> comparator) {
       this.fst = fst;
       this.topN = topN;
       this.fromNode = fromNode;
+      this.comparator = comparator;
     }
 
     // If back plus this arc is competitive then add to queue:
-    private void addIfCompetitive(FSTPath path) {
+    private void addIfCompetitive(FSTPath<T> path) {
 
       assert queue != null;
 
-      long cost = path.cost + path.arc.output;
+      T cost = fst.outputs.add(path.cost, path.arc.output);
       //System.out.println("  addIfCompetitive bottom=" + bottom + " queue.size()=" + queue.size());
 
       if (bottom != null) {
-
-        if (cost > bottom.cost) {
+        int comp = comparator.compare(cost, bottom.cost);
+        if (comp > 0) {
           // Doesn't compete
           return;
-        } else if (cost == bottom.cost) {
+        } else if (comp == 0) {
           // Tie break by alpha sort on the input:
           path.input.grow(path.input.length+1);
           path.input.ints[path.input.length++] = path.arc.label;
@@ -309,7 +308,7 @@ public final class Util {
         // Queue isn't full yet, so any path we hit competes:
       }
 
-      final FSTPath newPath = new FSTPath(cost, path.arc);
+      final FSTPath<T> newPath = new FSTPath<T>(cost, path.arc, comparator);
 
       newPath.input.grow(path.input.length+1);
       System.arraycopy(path.input.ints, 0, newPath.input.ints, 0, path.input.length);
@@ -319,7 +318,7 @@ public final class Util {
       //System.out.println("    add path=" + newPath);
       queue.add(newPath);
       if (bottom != null) {
-        final FSTPath removed = queue.pollLast();
+        final FSTPath<T> removed = queue.pollLast();
         assert removed == bottom;
         bottom = queue.last();
         //System.out.println("    now re-set bottom: " + bottom + " queue=" + queue);
@@ -330,13 +329,13 @@ public final class Util {
       }
     }
 
-    public MinResult[] search() throws IOException {
+    public MinResult<T>[] search() throws IOException {
       //System.out.println("  search topN=" + topN);
-      final FST.Arc<Long> scratchArc = new FST.Arc<Long>();
+      final FST.Arc<T> scratchArc = new FST.Arc<T>();
 
-      final List<MinResult> results = new ArrayList<MinResult>();
+      final List<MinResult<T>> results = new ArrayList<MinResult<T>>();
 
-      final Long NO_OUTPUT = fst.outputs.getNoOutput();
+      final T NO_OUTPUT = fst.outputs.getNoOutput();
 
       // TODO: we could enable FST to sorting arcs by weight
       // as it freezes... can easily do this on first pass
@@ -349,7 +348,7 @@ public final class Util {
       while (results.size() < topN) {
         //System.out.println("\nfind next path");
 
-        FSTPath path;
+        FSTPath<T> path;
 
         if (queue == null) {
 
@@ -360,20 +359,20 @@ public final class Util {
 
           // First pass (top path): start from original fromNode
           if (topN > 1) {
-            queue = new TreeSet<FSTPath>();
+            queue = new TreeSet<FSTPath<T>>();
           }
 
-          long minArcCost = Long.MAX_VALUE;
-          FST.Arc<Long> minArc = null;
+          T minArcCost = null;
+          FST.Arc<T> minArc = null;
 
-          path = new FSTPath(0, fromNode);
+          path = new FSTPath<T>(NO_OUTPUT, fromNode, comparator);
           fst.readFirstTargetArc(fromNode, path.arc);
 
           // Bootstrap: find the min starting arc
           while (true) {
-            long arcScore = path.arc.output;
+            T arcScore = path.arc.output;
             //System.out.println("  arc=" + (char) path.arc.label + " cost=" + arcScore);
-            if (arcScore < minArcCost) {
+            if (minArcCost == null || comparator.compare(arcScore, minArcCost) < 0) {
               minArcCost = arcScore;
               minArc = scratchArc.copyFrom(path.arc);
               //System.out.println("    **");
@@ -419,7 +418,7 @@ public final class Util {
           //System.out.println("    empty string!  cost=" + path.cost);
           // Empty string!
           path.input.length--;
-          results.add(new MinResult(path.input, path.cost));
+          results.add(new MinResult<T>(path.input, path.cost, comparator));
           continue;
         }
 
@@ -439,15 +438,16 @@ public final class Util {
         // For each input letter:
         while (true) {
 
-          //System.out.println("\n    cycle path: " + path);
-
+          //System.out.println("\n    cycle path: " + path);         
           fst.readFirstTargetArc(path.arc, path.arc);
 
           // For each arc leaving this node:
           boolean foundZero = false;
           while(true) {
             //System.out.println("      arc=" + (char) path.arc.label + " cost=" + path.arc.output);
-            if (path.arc.output == NO_OUTPUT) {
+            // tricky: instead of comparing output == 0, we must
+            // express it via the comparator compare(output, 0) == 0
+            if (comparator.compare(NO_OUTPUT, path.arc.output) == 0) {
               if (queue == null) {
                 foundZero = true;
                 break;
@@ -479,55 +479,53 @@ public final class Util {
           if (path.arc.label == FST.END_LABEL) {
             // Add final output:
             //System.out.println("    done!: " + path);
-            results.add(new MinResult(path.input, path.cost + path.arc.output));
+            results.add(new MinResult<T>(path.input, fst.outputs.add(path.cost, path.arc.output), comparator));
             break;
           } else {
             path.input.grow(1+path.input.length);
             path.input.ints[path.input.length] = path.arc.label;
             path.input.length++;
-            path.cost += path.arc.output;
+            path.cost = fst.outputs.add(path.cost, path.arc.output);
           }
         }
       }
     
-      return results.toArray(new MinResult[results.size()]);
+      @SuppressWarnings({"rawtypes","unchecked"}) final MinResult<T>[] arr =
+        (MinResult<T>[]) new MinResult[results.size()];
+      return results.toArray(arr);
     }
   }
 
-  // TODO: parameterize the FST type <T> and allow passing in a
-  // comparator; eg maybe your output is a PairOutput and
-  // one of the outputs in the pair is monotonic so you
-  // compare by that
-
-  public final static class MinResult implements Comparable<MinResult> {
+  public final static class MinResult<T> implements Comparable<MinResult<T>> {
     public final IntsRef input;
-    public final long output;
-    public MinResult(IntsRef input, long output) {
+    public final T output;
+    final Comparator<T> comparator;
+    public MinResult(IntsRef input, T output, Comparator<T> comparator) {
       this.input = input;
       this.output = output;
+      this.comparator = comparator;
     }
 
     @Override
-    public int compareTo(MinResult other) {
-      if (output < other.output) {
-        return -1;
-      } else if (output > other.output) {
-        return 1;
-      } else {
+    public int compareTo(MinResult<T> other) {
+      int cmp = comparator.compare(output, other.output);
+      if (cmp == 0) {
         return input.compareTo(other.input);
+      } else {
+        return cmp;
       }
     }
   }
 
-  /** Starting from node, find the top N min cost (Long
-   *  output) completions to a final node.
+  /** Starting from node, find the top N min cost 
+   * completions to a final node.
    *
    *  <p>NOTE: you must share the outputs when you build the
    *  FST (pass doShare=true to {@link
    *  PositiveIntOutputs#getSingleton}). */
 
-  public static MinResult[] shortestPaths(FST<Long> fst, FST.Arc<Long> fromNode, int topN) throws IOException {
-    return new TopNSearcher(fst, fromNode, topN).search();
+  public static <T> MinResult<T>[] shortestPaths(FST<T> fst, FST.Arc<T> fromNode, Comparator<T> comparator, int topN) throws IOException {
+    return new TopNSearcher<T>(fst, fromNode, topN, comparator).search();
   } 
 
   /**
@@ -639,7 +637,7 @@ public final class Util {
       while (!thisLevelQueue.isEmpty()) {
         final FST.Arc<T> arc = thisLevelQueue.remove(thisLevelQueue.size() - 1);
         //System.out.println("  pop: " + arc);
-        if (fst.targetHasArcs(arc)) {
+        if (FST.targetHasArcs(arc)) {
           // scan all target arcs
           //System.out.println("  readFirstTarget...");
           final int node = arc.target;
@@ -694,7 +692,7 @@ public final class Util {
               outs = "";
             }
 
-            if (!fst.targetHasArcs(arc) && arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) {
+            if (!FST.targetHasArcs(arc) && arc.isFinal() && arc.nextFinalOutput != NO_OUTPUT) {
               // Tricky special case: sometimes, due to
               // pruning, the builder can [sillily] produce
               // an FST with an arc into the final end state

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct16.java Tue Mar  6 23:17:08 2012
@@ -69,6 +69,7 @@ class Direct16 extends PackedInts.Reader
   }
 
   public long get(final int index) {
+    assert index >= 0 && index < size();
     return 0xFFFFL & values[index];
   }
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct32.java Tue Mar  6 23:17:08 2012
@@ -65,6 +65,7 @@ class Direct32 extends PackedInts.Reader
   }
 
   public long get(final int index) {
+    assert index >= 0 && index < size();
     return 0xFFFFFFFFL & values[index];
   }
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct64.java Tue Mar  6 23:17:08 2012
@@ -61,6 +61,7 @@ class Direct64 extends PackedInts.Reader
   }
 
   public long get(final int index) {
+    assert index >= 0 && index < size();
     return values[index];
   }
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Direct8.java Tue Mar  6 23:17:08 2012
@@ -70,6 +70,7 @@ class Direct8 extends PackedInts.ReaderI
   }
 
   public long get(final int index) {
+    assert index >= 0 && index < size();
     return 0xFFL & values[index];
   }
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed32.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed32.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed32.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed32.java Tue Mar  6 23:17:08 2012
@@ -186,6 +186,7 @@ class Packed32 extends PackedInts.Reader
    * @return the value at the given index.
    */
   public long get(final int index) {
+    assert index >= 0 && index < size();
     final long majorBitPos = (long)index * bitsPerValue;
     final int elementPos = (int)(majorBitPos >>> BLOCK_BITS); // / BLOCK_SIZE
     final int bitPos =     (int)(majorBitPos & MOD_MASK); // % BLOCK_SIZE);

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java Tue Mar  6 23:17:08 2012
@@ -176,6 +176,7 @@ class Packed64 extends PackedInts.Reader
    * @return the value at the given index.
    */
   public long get(final int index) {
+    assert index >= 0 && index < size();
     final long majorBitPos = (long)index * bitsPerValue;
     final int elementPos = (int)(majorBitPos >>> BLOCK_BITS); // / BLOCK_SIZE
     final int bitPos =     (int)(majorBitPos & MOD_MASK); // % BLOCK_SIZE);

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/TestExternalCodecs.java Tue Mar  6 23:17:08 2012
@@ -17,9 +17,6 @@ package org.apache.lucene;
  * limitations under the License.
  */
 
-import java.io.*;
-import java.util.*;
-
 import org.apache.lucene.analysis.*;
 import org.apache.lucene.codecs.*;
 import org.apache.lucene.codecs.lucene40.Lucene40Codec;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java Tue Mar  6 23:17:08 2012
@@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
 
@@ -76,7 +77,7 @@ public class TestCachingTokenFilter exte
                                                                           "preanalyzed",
                                                                           new BytesRef("term1"),
                                                                           false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(1, termPositions.freq());
     assertEquals(0, termPositions.nextPosition());
 
@@ -85,7 +86,7 @@ public class TestCachingTokenFilter exte
                                                      "preanalyzed",
                                                      new BytesRef("term2"),
                                                      false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(2, termPositions.freq());
     assertEquals(1, termPositions.nextPosition());
     assertEquals(3, termPositions.nextPosition());
@@ -95,7 +96,7 @@ public class TestCachingTokenFilter exte
                                                      "preanalyzed",
                                                      new BytesRef("term3"),
                                                      false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(1, termPositions.freq());
     assertEquals(2, termPositions.nextPosition());
     reader.close();

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java Tue Mar  6 23:17:08 2012
@@ -3,8 +3,6 @@ package org.apache.lucene.analysis;
 import java.io.StringReader;
 import java.util.Arrays;
 
-import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
-import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.BasicAutomata;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/analysis/tokenattributes/TestSimpleAttributeImpl.java Tue Mar  6 23:17:08 2012
@@ -29,6 +29,8 @@ public class TestSimpleAttributeImpl ext
   public void testAttributes() {
     _TestUtil.assertAttributeReflection(new PositionIncrementAttributeImpl(),
       Collections.singletonMap(PositionIncrementAttribute.class.getName()+"#positionIncrement", 1));
+    _TestUtil.assertAttributeReflection(new PositionLengthAttributeImpl(),
+      Collections.singletonMap(PositionLengthAttribute.class.getName()+"#positionLength", 1));
     _TestUtil.assertAttributeReflection(new FlagsAttributeImpl(),
       Collections.singletonMap(FlagsAttribute.class.getName()+"#flags", 0));
     _TestUtil.assertAttributeReflection(new TypeAttributeImpl(),

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/appending/TestAppendingCodec.java Tue Mar  6 23:17:08 2012
@@ -35,6 +35,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
@@ -141,10 +142,10 @@ public class TestAppendingCodec extends 
     assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("dog")));
     assertEquals(SeekStatus.FOUND, te.seekCeil(new BytesRef("the")));
     DocsEnum de = te.docs(null, null, true);
-    assertTrue(de.advance(0) != DocsEnum.NO_MORE_DOCS);
+    assertTrue(de.advance(0) != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(2, de.freq());
-    assertTrue(de.advance(1) != DocsEnum.NO_MORE_DOCS);
-    assertTrue(de.advance(2) == DocsEnum.NO_MORE_DOCS);
+    assertTrue(de.advance(1) != DocIdSetIterator.NO_MORE_DOCS);
+    assertTrue(de.advance(2) == DocIdSetIterator.NO_MORE_DOCS);
     reader.close();
   }
   

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestTermInfosReaderIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestTermInfosReaderIndex.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestTermInfosReaderIndex.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene3x/TestTermInfosReaderIndex.java Tue Mar  6 23:17:08 2012
@@ -25,12 +25,7 @@ import java.util.Random;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
-import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.FieldInfosReader;
-import org.apache.lucene.codecs.lucene3x.Lucene3xPostingsFormat;
-import org.apache.lucene.codecs.lucene3x.PreFlexRWCodec;
-import org.apache.lucene.codecs.lucene3x.SegmentTermEnum;
-import org.apache.lucene.codecs.lucene3x.TermInfosReaderIndex;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.CorruptIndexException;
@@ -86,8 +81,8 @@ public class TestTermInfosReaderIndex ex
     directory = newDirectory();
 
     config.setCodec(new PreFlexRWCodec());
-    // turn off compound file, this test will open some index files directly.
     LogMergePolicy mp = newLogMergePolicy();
+    // turn off compound file, this test will open some index files directly.
     mp.setUseCompoundFile(false);
     config.setMergePolicy(mp);
 
@@ -182,9 +177,16 @@ public class TestTermInfosReaderIndex ex
     int termPosition = index * termIndexInterval * indexDivisor;
     for (int i = 0; i < termPosition; i++) {
       // TODO: this test just uses random terms, so this is always possible
-      assumeTrue("ran out of terms.", termEnum.next());
+      assumeTrue("ran out of terms", termEnum.next());
     }
-    return termEnum.term();
+    final Term term = termEnum.term();
+    // An indexed term is only written when the term after
+    // it exists, so, if the number of terms is 0 mod
+    // termIndexInterval, the last index term will not be
+    // written; so we require a term after this term
+    // as well:
+    assumeTrue("ran out of terms", termEnum.next());
+    return term;
   }
 
   private static void populate(Directory directory, IndexWriterConfig config) throws CorruptIndexException, LockObtainFailedException, IOException {

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/codecs/lucene40/values/TestDocValues.java Tue Mar  6 23:17:08 2012
@@ -114,7 +114,7 @@ public class TestDocValues extends Lucen
           assertEquals("doc " + idx, values[idx], ss.getByOrd(ss.ord(idx),
               bytesRef).utf8ToString());
          int ord = ss
-              .getByValue(new BytesRef(values[idx]), new BytesRef());
+              .getOrdByValue(new BytesRef(values[idx]), new BytesRef());
           assertTrue(ord >= 0);
           assertEquals(ss.ord(idx), ord);
         }
@@ -125,7 +125,7 @@ public class TestDocValues extends Lucen
         final int valueCount = ss.getValueCount();
         for (int i = 0; i < 1000; i++) {
           BytesRef bytesValue = new BytesRef(_TestUtil.randomFixedByteLengthUnicodeString(random, fixedSize? fixedLength : 1 + random.nextInt(39)));
-          int ord = ss.getByValue(bytesValue, new BytesRef());
+          int ord = ss.getOrdByValue(bytesValue, new BytesRef());
           if (ord >= 0) {
             assertTrue(bytesValue
                 .bytesEquals(ss.getByOrd(ord, bytesRef)));

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/document/TestDateTools.java Tue Mar  6 23:17:08 2012
@@ -9,6 +9,10 @@ import java.util.TimeZone;
 import java.util.Locale;
 
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.SystemPropertiesRestoreRule;
+import org.junit.Rule;
+import org.junit.rules.RuleChain;
+import org.junit.rules.TestRule;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,6 +31,9 @@ import org.apache.lucene.util.LuceneTest
  * limitations under the License.
  */
 public class TestDateTools extends LuceneTestCase {
+  @Rule
+  public TestRule testRules = 
+    RuleChain.outerRule(new SystemPropertiesRestoreRule());
 
   public void testStringToDate() throws ParseException {
     

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestCodecs.java Tue Mar  6 23:17:08 2012
@@ -81,7 +81,7 @@ public class TestCodecs extends LuceneTe
     NUM_TEST_ITER = atLeast(20);
   }
 
-  class FieldData implements Comparable {
+  class FieldData implements Comparable<FieldData> {
     final FieldInfo fieldInfo;
     final TermData[] terms;
     final boolean omitTF;
@@ -102,8 +102,8 @@ public class TestCodecs extends LuceneTe
       Arrays.sort(terms);
     }
 
-    public int compareTo(final Object other) {
-      return fieldInfo.name.compareTo(((FieldData) other).fieldInfo.name);
+    public int compareTo(final FieldData other) {
+      return fieldInfo.name.compareTo(other.fieldInfo.name);
     }
 
     public void write(final FieldsConsumer consumer) throws Throwable {
@@ -133,7 +133,7 @@ public class TestCodecs extends LuceneTe
     }
   }
 
-  class TermData implements Comparable {
+  class TermData implements Comparable<TermData> {
     String text2;
     final BytesRef text;
     int[] docs;
@@ -147,8 +147,8 @@ public class TestCodecs extends LuceneTe
       this.positions = positions;
     }
 
-    public int compareTo(final Object o) {
-      return text.compareTo(((TermData) o).text);
+    public int compareTo(final TermData o) {
+      return text.compareTo(o.text);
     }
 
     public long write(final TermsConsumer termsConsumer) throws Throwable {
@@ -281,7 +281,7 @@ public class TestCodecs extends LuceneTe
       for(int iter=0;iter<2;iter++) {
         docsEnum = _TestUtil.docs(random, termsEnum, null,  docsEnum, false);
         assertEquals(terms[i].docs[0], docsEnum.nextDoc());
-        assertEquals(DocsEnum.NO_MORE_DOCS, docsEnum.nextDoc());
+        assertEquals(DocIdSetIterator.NO_MORE_DOCS, docsEnum.nextDoc());
       }
     }
     assertNull(termsEnum.next());
@@ -439,7 +439,7 @@ public class TestCodecs extends LuceneTe
         assertEquals(positions[i].pos, pos);
         if (positions[i].payload != null) {
           assertTrue(posEnum.hasPayload());
-          if (TestCodecs.random.nextInt(3) < 2) {
+          if (LuceneTestCase.random.nextInt(3) < 2) {
             // Verify the payload bytes
             final BytesRef otherPayload = posEnum.getPayload();
             assertTrue("expected=" + positions[i].payload.toString() + " got=" + otherPayload.toString(), positions[i].payload.equals(otherPayload));
@@ -453,7 +453,7 @@ public class TestCodecs extends LuceneTe
     public void _run() throws Throwable {
 
       for(int iter=0;iter<NUM_TEST_ITER;iter++) {
-        final FieldData field = fields[TestCodecs.random.nextInt(fields.length)];
+        final FieldData field = fields[LuceneTestCase.random.nextInt(fields.length)];
         final TermsEnum termsEnum = termsDict.terms(field.fieldInfo.name).iterator(null);
         if (si.getCodec() instanceof Lucene3xCodec) {
           // code below expects unicode sort order
@@ -473,7 +473,7 @@ public class TestCodecs extends LuceneTe
         assertEquals(upto, field.terms.length);
 
         // Test random seek:
-        TermData term = field.terms[TestCodecs.random.nextInt(field.terms.length)];
+        TermData term = field.terms[LuceneTestCase.random.nextInt(field.terms.length)];
         TermsEnum.SeekStatus status = termsEnum.seekCeil(new BytesRef(term.text2));
         assertEquals(status, TermsEnum.SeekStatus.FOUND);
         assertEquals(term.docs.length, termsEnum.docFreq());
@@ -484,7 +484,7 @@ public class TestCodecs extends LuceneTe
         }
 
         // Test random seek by ord:
-        final int idx = TestCodecs.random.nextInt(field.terms.length);
+        final int idx = LuceneTestCase.random.nextInt(field.terms.length);
         term = field.terms[idx];
         boolean success = false;
         try {
@@ -547,7 +547,7 @@ public class TestCodecs extends LuceneTe
         upto = 0;
         do {
           term = field.terms[upto];
-          if (TestCodecs.random.nextInt(3) == 1) {
+          if (LuceneTestCase.random.nextInt(3) == 1) {
             final DocsEnum docs;
             final DocsEnum docsAndFreqs;
             final DocsAndPositionsEnum postings;
@@ -569,10 +569,10 @@ public class TestCodecs extends LuceneTe
               // Maybe skip:
               final int left = term.docs.length-upto2;
               int doc;
-              if (TestCodecs.random.nextInt(3) == 1 && left >= 1) {
-                final int inc = 1+TestCodecs.random.nextInt(left-1);
+              if (LuceneTestCase.random.nextInt(3) == 1 && left >= 1) {
+                final int inc = 1+LuceneTestCase.random.nextInt(left-1);
                 upto2 += inc;
-                if (TestCodecs.random.nextInt(2) == 1) {
+                if (LuceneTestCase.random.nextInt(2) == 1) {
                   doc = docs.advance(term.docs[upto2]);
                   assertEquals(term.docs[upto2], doc);
                 } else {
@@ -597,7 +597,7 @@ public class TestCodecs extends LuceneTe
               assertEquals(term.docs[upto2], doc);
               if (!field.omitTF) {
                 assertEquals(term.positions[upto2].length, postings.freq());
-                if (TestCodecs.random.nextInt(2) == 1) {
+                if (LuceneTestCase.random.nextInt(2) == 1) {
                   this.verifyPositions(term.positions[upto2], postings);
                 }
               }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDirectoryReader.java Tue Mar  6 23:17:08 2012
@@ -110,7 +110,7 @@ public class TestDirectoryReader extends
 
     // This should blow up if we forget to check that the TermEnum is from the same
     // reader as the TermDocs.
-    while (td.nextDoc() != td.NO_MORE_DOCS) ret += td.docID();
+    while (td.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) ret += td.docID();
 
     // really a dummy assert to ensure that we got some docs and to ensure that
     // nothing is eliminated by hotspot

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocValuesIndexing.java Tue Mar  6 23:17:08 2012
@@ -23,16 +23,17 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.DocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DocValues.SortedSource;
@@ -792,7 +793,7 @@ public class TestDocValuesIndexing exten
         hash.get(sort[i], expected);
         asSortedSource.getByOrd(i, actual);
         assertEquals(expected.utf8ToString(), actual.utf8ToString());
-        int ord = asSortedSource.getByValue(expected, actual);
+        int ord = asSortedSource.getOrdByValue(expected, actual);
         assertEquals(i, ord);
       }
       AtomicReader slowR = SlowCompositeReaderWrapper.wrap(reader);
@@ -815,8 +816,113 @@ public class TestDocValuesIndexing exten
     assertEquals(1, docFreq);
     DocsEnum termDocsEnum = reader.termDocsEnum(null, term.field, term.bytes, false);
     int nextDoc = termDocsEnum.nextDoc();
-    assertEquals(DocsEnum.NO_MORE_DOCS, termDocsEnum.nextDoc());
+    assertEquals(DocIdSetIterator.NO_MORE_DOCS, termDocsEnum.nextDoc());
     return nextDoc;
+  }
+
+  public void testWithThreads() throws Exception {
+    final int NUM_DOCS = atLeast(100);
+    final Directory dir = newDirectory();
+    final RandomIndexWriter writer = new RandomIndexWriter(random, dir);
+    final boolean allowDups = random.nextBoolean();
+    final Set<String> seen = new HashSet<String>();
+    if (VERBOSE) {
+      System.out.println("TEST: NUM_DOCS=" + NUM_DOCS + " allowDups=" + allowDups);
+    }
+    int numDocs = 0;
+    final List<BytesRef> docValues = new ArrayList<BytesRef>();
+
+    // TODO: deletions
+    while (numDocs < NUM_DOCS) {
+      final String s;
+      if (random.nextBoolean()) {
+        s = _TestUtil.randomSimpleString(random);
+      } else {
+        s = _TestUtil.randomUnicodeString(random);
+      }
+      final BytesRef br = new BytesRef(s);
+
+      if (!allowDups) {
+        if (seen.contains(s)) {
+          continue;
+        }
+        seen.add(s);
+      }
+
+      if (VERBOSE) {
+        System.out.println("  " + numDocs + ": s=" + s);
+      }
+      
+      final Document doc = new Document();
+      doc.add(new DocValuesField("stringdv", br, DocValues.Type.BYTES_VAR_SORTED));
+      doc.add(new DocValuesField("id", numDocs, DocValues.Type.VAR_INTS));
+      docValues.add(br);
+      writer.addDocument(doc);
+      numDocs++;
+
+      if (random.nextInt(40) == 17) {
+        // force flush
+        writer.getReader().close();
+      }
+    }
+
+    writer.forceMerge(1);
+    final DirectoryReader r = writer.getReader();
+    writer.close();
     
+    final AtomicReader sr = getOnlySegmentReader(r);
+    final DocValues dv = sr.docValues("stringdv");
+    assertNotNull(dv);
+
+    final long END_TIME = System.currentTimeMillis() + (TEST_NIGHTLY ? 30 : 1);
+
+    final DocValues.Source docIDToID = sr.docValues("id").getSource();
+
+    final int NUM_THREADS = _TestUtil.nextInt(random, 1, 10);
+    Thread[] threads = new Thread[NUM_THREADS];
+    for(int thread=0;thread<NUM_THREADS;thread++) {
+      threads[thread] = new Thread() {
+          @Override
+          public void run() {
+            final DocValues.Source stringDVSource;
+            final DocValues.Source stringDVDirectSource;
+            try {
+              stringDVSource = dv.getSource();
+              assertNotNull(stringDVSource);
+              stringDVDirectSource = dv.getDirectSource();
+              assertNotNull(stringDVDirectSource);
+            } catch (IOException ioe) {
+              throw new RuntimeException(ioe);
+            }
+            while(System.currentTimeMillis() < END_TIME) {
+              final DocValues.Source source;
+              if (random.nextBoolean()) {
+                source = stringDVSource;
+              } else {
+                source = stringDVDirectSource;
+              }
+
+              final DocValues.SortedSource sortedSource = source.asSortedSource();
+              assertNotNull(sortedSource);
+
+              final BytesRef scratch = new BytesRef();
+
+              for(int iter=0;iter<100;iter++) {
+                final int docID = random.nextInt(sr.maxDoc());
+                final BytesRef br = sortedSource.getBytes(docID, scratch);
+                assertEquals(docValues.get((int) docIDToID.getInt(docID)), br);
+              }
+            }
+          }
+        };
+      threads[thread].start();
+    }
+
+    for(Thread thread : threads) {
+      thread.join();
+    }
+
+    r.close();
+    dir.close();
   }
 }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocsAndPositions.java Tue Mar  6 23:17:08 2012
@@ -84,7 +84,7 @@ public class TestDocsAndPositions extend
           assertEquals(msg, 20, docsAndPosEnum.nextPosition());
           assertEquals(msg, 4, docsAndPosEnum.freq());
           assertEquals(msg, 30, docsAndPosEnum.nextPosition());
-        } while (docsAndPosEnum.nextDoc() != DocsAndPositionsEnum.NO_MORE_DOCS);
+        } while (docsAndPosEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
       }
     }
     reader.close();
@@ -156,7 +156,7 @@ public class TestDocsAndPositions extend
         // now run through the scorer and check if all positions are there...
         do {
           int docID = docsAndPosEnum.docID();
-          if (docID == DocsAndPositionsEnum.NO_MORE_DOCS) {
+          if (docID == DocIdSetIterator.NO_MORE_DOCS) {
             break;
           }
           Integer[] pos = positionsInDoc[atomicReaderContext.docBase + docID];
@@ -177,7 +177,7 @@ public class TestDocsAndPositions extend
                 .advance(docID + 1 + random.nextInt((maxDoc - docID)));
           }
 
-        } while (docsAndPosEnum.nextDoc() != DocsAndPositionsEnum.NO_MORE_DOCS);
+        } while (docsAndPosEnum.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
       }
 
     }
@@ -234,7 +234,7 @@ public class TestDocsAndPositions extend
               int next = findNext(freqInDoc, context.docBase+j+1, context.docBase + maxDoc) - context.docBase;
               int advancedTo = docsEnum.advance(next);
               if (next >= maxDoc) {
-                assertEquals(DocsEnum.NO_MORE_DOCS, advancedTo);
+                assertEquals(DocIdSetIterator.NO_MORE_DOCS, advancedTo);
               } else {
                 assertTrue("advanced to: " +advancedTo + " but should be <= " + next, next >= advancedTo);  
               }
@@ -243,7 +243,7 @@ public class TestDocsAndPositions extend
             }
           } 
         }
-        assertEquals("docBase: " + context.docBase + " maxDoc: " + maxDoc + " " + docsEnum.getClass(), DocsEnum.NO_MORE_DOCS, docsEnum.docID());
+        assertEquals("docBase: " + context.docBase + " maxDoc: " + maxDoc + " " + docsEnum.getClass(), DocIdSetIterator.NO_MORE_DOCS, docsEnum.docID());
       }
       
     }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java Tue Mar  6 23:17:08 2012
@@ -30,6 +30,7 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
@@ -129,7 +130,7 @@ public class TestDocumentWriter extends 
 
     DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getLiveDocs(reader),
                                                                           "repeated", new BytesRef("repeated"), false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     int freq = termPositions.freq();
     assertEquals(2, freq);
     assertEquals(0, termPositions.nextPosition());
@@ -200,7 +201,7 @@ public class TestDocumentWriter extends 
     SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
 
     DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, reader.getLiveDocs(), "f1", new BytesRef("a"), false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     int freq = termPositions.freq();
     assertEquals(3, freq);
     assertEquals(0, termPositions.nextPosition());
@@ -244,18 +245,18 @@ public class TestDocumentWriter extends 
     SegmentReader reader = new SegmentReader(info, DirectoryReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
 
     DocsAndPositionsEnum termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term1"), false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(1, termPositions.freq());
     assertEquals(0, termPositions.nextPosition());
 
     termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term2"), false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(2, termPositions.freq());
     assertEquals(1, termPositions.nextPosition());
     assertEquals(3, termPositions.nextPosition());
     
     termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term3"), false);
-    assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
+    assertTrue(termPositions.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(1, termPositions.freq());
     assertEquals(2, termPositions.nextPosition());
     reader.close();

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/test/org/apache/lucene/index/TestDuelingCodecs.java Tue Mar  6 23:17:08 2012
@@ -18,10 +18,13 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Random;
 import java.util.Set;
+import java.util.TreeSet;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
@@ -34,7 +37,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.automaton.AutomatonTestUtil;
 import org.apache.lucene.util.automaton.CompiledAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
@@ -96,9 +98,9 @@ public class TestDuelingCodecs extends L
     createRandomIndex(numdocs, leftWriter, seed);
     createRandomIndex(numdocs, rightWriter, seed);
 
-    leftReader = leftWriter.getReader();
+    leftReader = maybeWrapReader(leftWriter.getReader());
     leftWriter.close();
-    rightReader = rightWriter.getReader();
+    rightReader = maybeWrapReader(rightWriter.getReader());
     rightWriter.close();
     
     info = "left: " + leftCodec.toString() + " / right: " + rightCodec.toString();
@@ -140,6 +142,7 @@ public class TestDuelingCodecs extends L
     assertTermVectors(leftReader, rightReader);
     assertDocValues(leftReader, rightReader);
     assertDeletedDocs(leftReader, rightReader);
+    assertFieldInfos(leftReader, rightReader);
   }
   
   /** 
@@ -479,8 +482,18 @@ public class TestDuelingCodecs extends L
       Document rightDoc = rightReader.document(i);
       
       // TODO: I think this is bogus because we don't document what the order should be
-      // from these iterators, etc. I think the codec should be free to order this stuff
+      // from these iterators, etc. I think the codec/IndexReader should be free to order this stuff
       // in whatever way it wants (e.g. maybe it packs related fields together or something)
+      // To fix this, we sort the fields in both documents by name, but
+      // we still assume that all instances with same name are in order:
+      Comparator<IndexableField> comp = new Comparator<IndexableField>() {
+        @Override
+        public int compare(IndexableField arg0, IndexableField arg1) {
+          return arg0.name().compareTo(arg1.name());
+        }        
+      };
+      Collections.sort(leftDoc.getFields(), comp);
+      Collections.sort(rightDoc.getFields(), comp);
 
       Iterator<IndexableField> leftIterator = leftDoc.iterator();
       Iterator<IndexableField> rightIterator = rightDoc.iterator();
@@ -614,6 +627,25 @@ public class TestDuelingCodecs extends L
     }
   }
   
+  public void assertFieldInfos(IndexReader leftReader, IndexReader rightReader) throws Exception {
+    FieldInfos leftInfos = MultiFields.getMergedFieldInfos(leftReader);
+    FieldInfos rightInfos = MultiFields.getMergedFieldInfos(rightReader);
+    
+    // TODO: would be great to verify more than just the names of the fields!
+    TreeSet<String> left = new TreeSet<String>();
+    TreeSet<String> right = new TreeSet<String>();
+    
+    for (FieldInfo fi : leftInfos) {
+      left.add(fi.name);
+    }
+    
+    for (FieldInfo fi : rightInfos) {
+      right.add(fi.name);
+    }
+    
+    assertEquals(info, left, right);
+  }
+  
   
   private static class RandomBits implements Bits {
     FixedBitSet bits;