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 [3/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/codecs/TermsIndexReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/TermsIndexReaderBase.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/TermsIndexReaderBase.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/TermsIndexReaderBase.java Tue Mar  6 23:17:08 2012
@@ -22,7 +22,6 @@ import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
 import java.io.Closeable;
-import java.util.Collection;
 
 
 // TODO

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexWriter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexWriter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/VariableGapTermsIndexWriter.java Tue Mar  6 23:17:08 2012
@@ -54,7 +54,8 @@ public class VariableGapTermsIndexWriter
   final static int VERSION_CURRENT = VERSION_START;
 
   private final List<FSTFieldWriter> fields = new ArrayList<FSTFieldWriter>();
-  private final FieldInfos fieldInfos; // unread
+  
+  @SuppressWarnings("unused") private final FieldInfos fieldInfos; // unread
   private final IndexTermSelector policy;
 
   /** @lucene.experimental */
@@ -214,7 +215,6 @@ public class VariableGapTermsIndexWriter
     private final long startTermsFilePointer;
 
     final FieldInfo fieldInfo;
-    int numIndexTerms;
     FST<Long> fst;
     final long indexStart;
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/appending/AppendingPostingsFormat.java Tue Mar  6 23:17:08 2012
@@ -32,7 +32,6 @@ import org.apache.lucene.codecs.lucene40
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.store.Directory;
 
 /**
  * Appending postings impl

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/codecs/lucene40/values/VarSortedBytesImpl.java Tue Mar  6 23:17:08 2012
@@ -63,6 +63,7 @@ final class VarSortedBytesImpl {
       this.comp = comp;
       size = 0;
     }
+
     @Override
     public void merge(MergeState mergeState, DocValues[] docValues)
         throws IOException {

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

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/AtomicReader.java Tue Mar  6 23:17:08 2012
@@ -20,7 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.search.SearcherManager; // javadocs
-import org.apache.lucene.store.*;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ReaderUtil;         // for javadocs

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/BufferedDeletesStream.java Tue Mar  6 23:17:08 2012
@@ -401,7 +401,7 @@ class BufferedDeletesStream {
           while (true) {
             final int docID = docsEnum.nextDoc();
             //System.out.println(Thread.currentThread().getName() + " del term=" + term + " doc=" + docID);
-            if (docID == DocsEnum.NO_MORE_DOCS) {
+            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }   
             // NOTE: there is no limit check on the docID

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Tue Mar  6 23:17:08 2012
@@ -576,7 +576,7 @@ public class CheckIndex {
         segInfoStat.fieldNormStatus = testFieldNorms(fieldInfos, reader);
 
         // Test the Term Index
-        segInfoStat.termIndexStatus = testPostings(reader);
+        segInfoStat.termIndexStatus = testPostings(fieldInfos, reader);
 
         // Test Stored Fields
         segInfoStat.storedFieldStatus = testStoredFields(info, reader, nf);
@@ -691,7 +691,7 @@ public class CheckIndex {
   /**
    * Test the term index.
    */
-  private Status.TermIndexStatus testPostings(SegmentReader reader) {
+  private Status.TermIndexStatus testPostings(FieldInfos fieldInfos, SegmentReader reader) {
 
     // TODO: we should go and verify term vectors match, if
     // crossCheckTermVectors is on...
@@ -720,15 +720,31 @@ public class CheckIndex {
       DocsEnum docsAndFreqs = null;
       DocsAndPositionsEnum postings = null;
 
+      String lastField = null;
       final FieldsEnum fieldsEnum = fields.iterator();
       while(true) {
         final String field = fieldsEnum.next();
         if (field == null) {
           break;
         }
+        // MultiFieldsEnum relies upon this order...
+        if (lastField != null && field.compareTo(lastField) <= 0) {
+          throw new RuntimeException("fields out of order: lastField=" + lastField + " field=" + field);
+        }
+        lastField = field;
+        
+        // check that the field is in fieldinfos, and is indexed.
+        // TODO: add a separate test to check this for different reader impls
+        FieldInfo fi = fieldInfos.fieldInfo(field);
+        if (fi == null) {
+          throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, no fieldInfos for: " + field);
+        }
+        if (!fi.isIndexed) {
+          throw new RuntimeException("fieldsEnum inconsistent with fieldInfos, isIndexed == false for: " + field);
+        }
 
         // TODO: really the codec should not return a field
-        // from FieldsEnum if it has to Terms... but we do
+        // from FieldsEnum if it has no Terms... but we do
         // this today:
         // assert fields.terms(field) != null;
         computedFieldCount++;
@@ -909,7 +925,7 @@ public class CheckIndex {
               final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
               postings = termsEnum.docsAndPositions(liveDocs, postings, false);
               final int docID = postings.advance(skipDocID);
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               } else {
                 if (docID < skipDocID) {
@@ -932,7 +948,7 @@ public class CheckIndex {
                 } 
 
                 final int nextDocID = postings.nextDoc();
-                if (nextDocID == DocsEnum.NO_MORE_DOCS) {
+                if (nextDocID == DocIdSetIterator.NO_MORE_DOCS) {
                   break;
                 }
                 if (nextDocID <= docID) {
@@ -945,14 +961,14 @@ public class CheckIndex {
               final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
               docs = termsEnum.docs(liveDocs, docs, false);
               final int docID = docs.advance(skipDocID);
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               } else {
                 if (docID < skipDocID) {
                   throw new RuntimeException("term " + term + ": advance(docID=" + skipDocID + ") returned docID=" + docID);
                 }
                 final int nextDocID = docs.nextDoc();
-                if (nextDocID == DocsEnum.NO_MORE_DOCS) {
+                if (nextDocID == DocIdSetIterator.NO_MORE_DOCS) {
                   break;
                 }
                 if (nextDocID <= docID) {
@@ -1051,7 +1067,7 @@ public class CheckIndex {
                   throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
                 }
 
-                while(docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+                while(docs.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
                   totDocCount++;
                 }
               }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ConcurrentMergeScheduler.java Tue Mar  6 23:17:08 2012
@@ -488,19 +488,6 @@ public class ConcurrentMergeScheduler ex
         }
       }
     }
-
-    @Override
-    public String toString() {
-      MergePolicy.OneMerge merge = getRunningMerge();
-      if (merge == null) {
-        merge = startMerge;
-      }
-      try {
-        return "merge thread: " + tWriter.segString(merge.segments);
-      } catch (IOException ioe) {
-        throw new RuntimeException(ioe);
-      }
-    }
   }
 
   /** Called when an exception is hit in a background merge

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DirectoryReader.java Tue Mar  6 23:17:08 2012
@@ -48,13 +48,8 @@ import org.apache.lucene.store.Directory
  synchronization, you should <b>not</b> synchronize on the
  <code>IndexReader</code> instance; use your own
  (non-Lucene) objects instead.
- 
- <p><em>Please note:</em> This class extends from an internal (invisible)
- superclass that is generic: The type parameter {@code R} is
- {@link AtomicReader}, see {@link #subReaders} and
- {@link #getSequentialSubReaders}.
 */
-public abstract class DirectoryReader extends BaseMultiReader<AtomicReader> {
+public abstract class DirectoryReader extends BaseCompositeReader<AtomicReader> {
   public static final int DEFAULT_TERMS_INDEX_DIVISOR = 1;
 
   protected final Directory directory;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocFieldProcessor.java Tue Mar  6 23:17:08 2012
@@ -337,6 +337,9 @@ final class DocFieldProcessor extends Do
     if (perDocConsumer == null) {
       PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
       perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState);
+      if (perDocConsumer == null) {
+        throw new IllegalStateException("codec=" +  docState.docWriter.codec + " does not support docValues: from docValuesFormat().docsConsumer(...) returned null; field=" + fieldInfo.name);
+      }
     }
     DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
     fieldInfo.setDocValuesType(valueType, false);

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocTermOrds.java Tue Mar  6 23:17:08 2012
@@ -642,19 +642,17 @@ public class DocTermOrds {
    * ord; in this case we "wrap" our own terms index
    * around it. */
   private final class OrdWrappedTermsEnum extends TermsEnum {
-    private final AtomicReader reader;
     private final TermsEnum termsEnum;
     private BytesRef term;
     private long ord = -indexInterval-1;          // force "real" seek
     
     public OrdWrappedTermsEnum(AtomicReader reader) throws IOException {
-      this.reader = reader;
       assert indexedTermsArray != null;
       termsEnum = reader.fields().terms(field).iterator(null);
     }
 
     @Override
-    public Comparator<BytesRef> getComparator() throws IOException {
+    public Comparator<BytesRef> getComparator() {
       return termsEnum.getComparator();
     }
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocValues.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocValues.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/DocValues.java Tue Mar  6 23:17:08 2012
@@ -148,6 +148,7 @@ public abstract class DocValues implemen
     protected Source(Type type) {
       this.type = type;
     }
+
     /**
      * Returns a <tt>long</tt> for the given document id or throws an
      * {@link UnsupportedOperationException} if this source doesn't support
@@ -239,9 +240,10 @@ public abstract class DocValues implemen
     public BytesRef getBytes(int docID, BytesRef bytesRef) {
       final int ord = ord(docID);
       if (ord < 0) {
+        // Negative ord means doc was missing?
         bytesRef.length = 0;
       } else {
-        getByOrd(ord , bytesRef);
+        getByOrd(ord, bytesRef);
       }
       return bytesRef;
     }
@@ -253,7 +255,7 @@ public abstract class DocValues implemen
     public abstract int ord(int docID);
 
     /** Returns value for specified ord. */
-    public abstract BytesRef getByOrd(int ord, BytesRef bytesRef);
+    public abstract BytesRef getByOrd(int ord, BytesRef result);
 
     /** Return true if it's safe to call {@link
      *  #getDocToOrd}. */
@@ -274,7 +276,7 @@ public abstract class DocValues implemen
     }
 
     /**
-     * Performs a lookup by value.
+     * Lookup ord by value.
      * 
      * @param value
      *          the value to look up
@@ -283,11 +285,11 @@ public abstract class DocValues implemen
      *          values to the given value. Must not be <code>null</code>
      * @return the given values ordinal if found or otherwise
      *         <code>(-(ord)-1)</code>, defined as the ordinal of the first
-     *         element that is greater than the given value. This guarantees
-     *         that the return value will always be &gt;= 0 if the given value
-     *         is found.
+     *         element that is greater than the given value (the insertion
+     *         point). This guarantees that the return value will always be
+     *         &gt;= 0 if the given value is found.
      */
-    public int getByValue(BytesRef value, BytesRef spare) {
+    public int getOrdByValue(BytesRef value, BytesRef spare) {
       return binarySearch(value, spare, 0, getValueCount() - 1);
     }    
 
@@ -405,7 +407,7 @@ public abstract class DocValues implemen
       }
 
       @Override
-      public int getByValue(BytesRef value, BytesRef spare) {
+      public int getOrdByValue(BytesRef value, BytesRef spare) {
         if (value.length == 0) {
           return 0;
         } else {
@@ -414,7 +416,7 @@ public abstract class DocValues implemen
       }
 
       @Override
-        public int getValueCount() {
+      public int getValueCount() {
         return 1;
       }
     };

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/FilterAtomicReader.java Tue Mar  6 23:17:08 2012
@@ -17,8 +17,10 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 import java.io.IOException;
 import java.util.Comparator;
@@ -26,7 +28,7 @@ import java.util.Comparator;
 /**  A <code>FilterAtomicReader</code> contains another AtomicReader, which it
  * uses as its basic source of data, possibly transforming the data along the
  * way or providing additional functionality. The class
- * <code>FilterIndexReader</code> itself simply implements all abstract methods
+ * <code>FilterAtomicReader</code> itself simply implements all abstract methods
  * of <code>IndexReader</code> with versions that pass all requests to the
  * contained index reader. Subclasses of <code>FilterAtomicReader</code> may
  * further override some of these methods and may also provide additional
@@ -37,7 +39,7 @@ public class FilterAtomicReader extends 
   /** Base class for filtering {@link Fields}
    *  implementations. */
   public static class FilterFields extends Fields {
-    protected Fields in;
+    protected final Fields in;
 
     public FilterFields(Fields in) {
       this.in = in;
@@ -57,12 +59,17 @@ public class FilterAtomicReader extends 
     public int getUniqueFieldCount() throws IOException {
       return in.getUniqueFieldCount();
     }
+
+    @Override
+    public long getUniqueTermCount() throws IOException {
+      return in.getUniqueTermCount();
+    }
   }
 
   /** Base class for filtering {@link Terms}
    *  implementations. */
   public static class FilterTerms extends Terms {
-    protected Terms in;
+    protected final Terms in;
 
     public FilterTerms(Terms in) {
       this.in = in;
@@ -97,11 +104,16 @@ public class FilterAtomicReader extends 
     public int getDocCount() throws IOException {
       return in.getDocCount();
     }
+    
+    @Override
+    public TermsEnum intersect(CompiledAutomaton automaton, BytesRef bytes) throws java.io.IOException {
+      return in.intersect(automaton, bytes);
+    }
   }
 
   /** Base class for filtering {@link TermsEnum} implementations. */
   public static class FilterFieldsEnum extends FieldsEnum {
-    protected FieldsEnum in;
+    protected final FieldsEnum in;
     public FilterFieldsEnum(FieldsEnum in) {
       this.in = in;
     }
@@ -115,11 +127,16 @@ public class FilterAtomicReader extends 
     public Terms terms() throws IOException {
       return in.terms();
     }
+    
+    @Override
+    public AttributeSource attributes() {
+      return in.attributes();
+    }
   }
 
   /** Base class for filtering {@link TermsEnum} implementations. */
   public static class FilterTermsEnum extends TermsEnum {
-    protected TermsEnum in;
+    protected final TermsEnum in;
 
     public FilterTermsEnum(TermsEnum in) { this.in = in; }
 
@@ -174,7 +191,7 @@ public class FilterAtomicReader extends 
     }
 
     @Override
-    public Comparator<BytesRef> getComparator() throws IOException {
+    public Comparator<BytesRef> getComparator() {
       return in.getComparator();
     }
 
@@ -187,11 +204,16 @@ public class FilterAtomicReader extends 
     public TermState termState() throws IOException {
       return in.termState();
     }
+    
+    @Override
+    public AttributeSource attributes() {
+      return in.attributes();
+    }
   }
 
   /** Base class for filtering {@link DocsEnum} implementations. */
   public static class FilterDocsEnum extends DocsEnum {
-    protected DocsEnum in;
+    protected final DocsEnum in;
 
     public FilterDocsEnum(DocsEnum in) {
       this.in = in;
@@ -216,11 +238,16 @@ public class FilterAtomicReader extends 
     public int advance(int target) throws IOException {
       return in.advance(target);
     }
+    
+    @Override
+    public AttributeSource attributes() {
+      return in.attributes();
+    }
   }
 
   /** Base class for filtering {@link DocsAndPositionsEnum} implementations. */
   public static class FilterDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    protected DocsAndPositionsEnum in;
+    protected final DocsAndPositionsEnum in;
 
     public FilterDocsAndPositionsEnum(DocsAndPositionsEnum in) {
       this.in = in;
@@ -270,18 +297,24 @@ public class FilterAtomicReader extends 
     public boolean hasPayload() {
       return in.hasPayload();
     }
+    
+    @Override
+    public AttributeSource attributes() {
+      return in.attributes();
+    }
   }
 
-  protected AtomicReader in;
+  protected final AtomicReader in;
 
   /**
-   * <p>Construct a FilterIndexReader based on the specified base reader.
-   * <p>Note that base reader is closed if this FilterIndexReader is closed.</p>
+   * <p>Construct a FilterAtomicReader based on the specified base reader.
+   * <p>Note that base reader is closed if this FilterAtomicReader is closed.</p>
    * @param in specified base reader.
    */
   public FilterAtomicReader(AtomicReader in) {
     super();
     this.in = in;
+    in.registerParentReader(this);
   }
 
   @Override
@@ -363,7 +396,7 @@ public class FilterAtomicReader extends 
 
   @Override
   public String toString() {
-    final StringBuilder buffer = new StringBuilder("FilterIndexReader(");
+    final StringBuilder buffer = new StringBuilder("FilterAtomicReader(");
     buffer.append(in);
     buffer.append(')');
     return buffer.toString();

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

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexFileNameFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexFileNameFilter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexFileNameFilter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexFileNameFilter.java Tue Mar  6 23:17:08 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import java.io.File;
 import java.io.FilenameFilter;
-import java.util.HashSet;
 import java.util.regex.Pattern;
 
 /**

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexReader.java Tue Mar  6 23:17:08 2012
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.LinkedHashSet;
+import java.util.WeakHashMap;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -72,10 +73,13 @@ import org.apache.lucene.util.ReaderUtil
 */
 public abstract class IndexReader implements Closeable {
   
+  private boolean closed = false;
+  private boolean closedByChild = false;
+  private final AtomicInteger refCount = new AtomicInteger(1);
+
   IndexReader() {
     if (!(this instanceof CompositeReader || this instanceof AtomicReader))
-      throw new Error("This class should never be directly extended, subclass AtomicReader or CompositeReader instead!");
-    refCount.set(1);
+      throw new Error("IndexReader should never be directly extended, subclass AtomicReader or CompositeReader instead.");
   }
   
   /**
@@ -91,6 +95,9 @@ public abstract class IndexReader implem
   private final Set<ReaderClosedListener> readerClosedListeners = 
       Collections.synchronizedSet(new LinkedHashSet<ReaderClosedListener>());
 
+  private final Set<IndexReader> parentReaders = 
+      Collections.synchronizedSet(Collections.newSetFromMap(new WeakHashMap<IndexReader,Boolean>()));
+
   /** Expert: adds a {@link ReaderClosedListener}.  The
    * provided listener will be invoked when this reader is closed.
    *
@@ -107,8 +114,19 @@ public abstract class IndexReader implem
     ensureOpen();
     readerClosedListeners.remove(listener);
   }
+  
+  /** Expert: This method is called by {@code IndexReader}s which wrap other readers
+   * (e.g. {@link CompositeReader} or {@link FilterAtomicReader}) to register the parent
+   * at the child (this reader) on construction of the parent. When this reader is closed,
+   * it will mark all registered parents as closed, too. The references to parent readers
+   * are weak only, so they can be GCed once they are no longer in use.
+   * @lucene.experimental */
+  public final void registerParentReader(IndexReader reader) {
+    ensureOpen();
+    parentReaders.add(reader);
+  }
 
-  private final void notifyReaderClosedListeners() {
+  private void notifyReaderClosedListeners() {
     synchronized(readerClosedListeners) {
       for(ReaderClosedListener listener : readerClosedListeners) {
         listener.onClose(this);
@@ -116,9 +134,17 @@ public abstract class IndexReader implem
     }
   }
 
-  private boolean closed = false;
-  
-  private final AtomicInteger refCount = new AtomicInteger();
+  private void reportCloseToParentReaders() {
+    synchronized(parentReaders) {
+      for(IndexReader parent : parentReaders) {
+        parent.closedByChild = true;
+        // cross memory barrier by a fake write:
+        parent.refCount.addAndGet(0);
+        // recurse:
+        parent.reportCloseToParentReaders();
+      }
+    }
+  }
 
   /** Expert: returns the current refCount for this reader */
   public final int getRefCount() {
@@ -191,7 +217,12 @@ public abstract class IndexReader implem
    * @see #incRef
    */
   public final void decRef() throws IOException {
-    ensureOpen();
+    // only check refcount here (don't call ensureOpen()), so we can
+    // still close the reader if it was made invalid by a child:
+    if (refCount.get() <= 0) {
+      throw new AlreadyClosedException("this IndexReader is closed");
+    }
+    
     final int rc = refCount.decrementAndGet();
     if (rc == 0) {
       boolean success = false;
@@ -204,6 +235,7 @@ public abstract class IndexReader implem
           refCount.incrementAndGet();
         }
       }
+      reportCloseToParentReaders();
       notifyReaderClosedListeners();
     } else if (rc < 0) {
       throw new IllegalStateException("too many decRef calls: refCount is " + rc + " after decrement");
@@ -217,6 +249,33 @@ public abstract class IndexReader implem
     if (refCount.get() <= 0) {
       throw new AlreadyClosedException("this IndexReader is closed");
     }
+    // the happens before rule on reading the refCount, which must be after the fake write,
+    // ensures that we see the value:
+    if (closedByChild) {
+      throw new AlreadyClosedException("this IndexReader cannot be used anymore as one of its child readers was closed");
+    }
+  }
+  
+  /** {@inheritDoc}
+   * <p>For caching purposes, {@code IndexReader} subclasses are not allowed
+   * to implement equals/hashCode, so methods are declared final.
+   * To lookup instances from caches use {@link #getCoreCacheKey} and 
+   * {@link #getCombinedCoreAndDeletesKey}.
+   */
+  @Override
+  public final boolean equals(Object obj) {
+    return (this == obj);
+  }
+  
+  /** {@inheritDoc}
+   * <p>For caching purposes, {@code IndexReader} subclasses are not allowed
+   * to implement equals/hashCode, so methods are declared final.
+   * To lookup instances from caches use {@link #getCoreCacheKey} and 
+   * {@link #getCombinedCoreAndDeletesKey}.
+   */
+  @Override
+  public final int hashCode() {
+    return System.identityHashCode(this);
   }
   
   /** Returns a IndexReader reading the index in the given

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java Tue Mar  6 23:17:08 2012
@@ -532,7 +532,7 @@ public class IndexWriter implements Clos
 
     public synchronized boolean delete(int docID) {
       assert liveDocs != null;
-      assert docID >= 0 && docID < liveDocs.length();
+      assert docID >= 0 && docID < liveDocs.length() : "out of bounds: docid=" + docID + ",liveDocsLength=" + liveDocs.length();
       assert !shared;
       final boolean didDelete = liveDocs.get(docID);
       if (didDelete) {
@@ -577,6 +577,7 @@ public class IndexWriter implements Clos
 
     public synchronized void initWritableLiveDocs() throws IOException {
       assert Thread.holdsLock(IndexWriter.this);
+      assert info.docCount > 0;
       //System.out.println("initWritableLivedocs seg=" + info + " liveDocs=" + liveDocs + " shared=" + shared);
       if (shared) {
         // Copy on write: this means we've cloned a
@@ -3133,7 +3134,7 @@ public class IndexWriter implements Clos
       return false;
     }
 
-    final ReadersAndLiveDocs mergedDeletes = commitMergedDeletes(merge);
+    final ReadersAndLiveDocs mergedDeletes =  merge.info.docCount == 0 ? null : commitMergedDeletes(merge);
 
     assert mergedDeletes == null || mergedDeletes.pendingDeleteCount != 0;
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Tue Mar  6 23:17:08 2012
@@ -35,7 +35,7 @@ import org.apache.lucene.util.ReaderUtil
  * Exposes flex API, merged from flex API of sub-segments.
  * This is useful when you're interacting with an {@link
  * IndexReader} implementation that consists of sequential
- * sub-readers (eg DirectoryReader or {@link
+ * sub-readers (eg {@link DirectoryReader} or {@link
  * MultiReader}).
  *
  * <p><b>NOTE</b>: for multi readers, you'll get better

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/MultiReader.java Tue Mar  6 23:17:08 2012
@@ -19,15 +19,25 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-/** An IndexReader which reads multiple indexes, appending
- *  their content.
- 
- <p><em>Please note:</em> This class extends from an internal (invisible)
- superclass that is generic: The type parameter {@code R} is
- {@link IndexReader}, see {@link #subReaders} and
- {@link #getSequentialSubReaders}.
+/** A {@link CompositeReader} which reads multiple indexes, appending
+ *  their content. It can be used to create a view on several
+ *  sub-readers (like {@link DirectoryReader}) and execute searches on it.
+ * 
+ * <p> For efficiency, in this API documents are often referred to via
+ * <i>document numbers</i>, non-negative integers which each name a unique
+ * document in the index.  These document numbers are ephemeral -- they may change
+ * as documents are added to and deleted from an index.  Clients should thus not
+ * rely on a given document having the same number between sessions.
+ * 
+ * <p><a name="thread-safety"></a><p><b>NOTE</b>: {@link
+ * IndexReader} instances are completely thread
+ * safe, meaning multiple threads can call any of its methods,
+ * concurrently.  If your application requires external
+ * synchronization, you should <b>not</b> synchronize on the
+ * <code>IndexReader</code> instance; use your own
+ * (non-Lucene) objects instead.
  */
-public class MultiReader extends BaseMultiReader<IndexReader> {
+public class MultiReader extends BaseCompositeReader<IndexReader> {
   private final boolean closeSubReaders;
   
  /**

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/NormsConsumerPerField.java Tue Mar  6 23:17:08 2012
@@ -18,11 +18,8 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.codecs.DocValuesConsumer;
-import org.apache.lucene.document.DocValuesField;
-import org.apache.lucene.document.Field;
 import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.BytesRef;
 
 public class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
   private final FieldInfo fieldInfo;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelAtomicReader.java Tue Mar  6 23:17:08 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -57,7 +56,8 @@ public final class ParallelAtomicReader 
   private final boolean closeSubReaders;
   private final int maxDoc, numDocs;
   private final boolean hasDeletions;
-  final SortedMap<String,AtomicReader> fieldToReader = new TreeMap<String,AtomicReader>();
+  private final SortedMap<String,AtomicReader> fieldToReader = new TreeMap<String,AtomicReader>();
+  private final SortedMap<String,AtomicReader> tvFieldToReader = new TreeMap<String,AtomicReader>();
   
   /** Create a ParallelAtomicReader based on the provided
    *  readers; auto-closes the given readers on {@link #close()}. */
@@ -98,24 +98,43 @@ public final class ParallelAtomicReader 
         throw new IllegalArgumentException("All readers must have same maxDoc: "+maxDoc+"!="+reader.maxDoc());
       }
     }
-      
+    
+    // build FieldInfos and fieldToReader map:
     for (final AtomicReader reader : this.parallelReaders) {
       final FieldInfos readerFieldInfos = reader.getFieldInfos();
-      for(FieldInfo fieldInfo : readerFieldInfos) { // update fieldToReader map
+      for (FieldInfo fieldInfo : readerFieldInfos) {
         // NOTE: first reader having a given field "wins":
         if (!fieldToReader.containsKey(fieldInfo.name)) {
           fieldInfos.add(fieldInfo);
           fieldToReader.put(fieldInfo.name, reader);
-          this.fields.addField(fieldInfo.name, reader.terms(fieldInfo.name));
+          if (fieldInfo.storeTermVector) {
+            tvFieldToReader.put(fieldInfo.name, reader);
+          }
+        }
+      }
+    }
+    
+    // build Fields instance
+    for (final AtomicReader reader : this.parallelReaders) {
+      final Fields readerFields = reader.fields();
+      if (readerFields != null) {
+        final FieldsEnum it = readerFields.iterator();
+        String name;
+        while ((name = it.next()) != null) {
+          // only add if the reader responsible for that field name is the current:
+          if (fieldToReader.get(name) == reader) {
+            this.fields.addField(name, it.terms());
+          }
         }
       }
-    } 
+    }
 
     // do this finally so any Exceptions occurred before don't affect refcounts:
-    if (!closeSubReaders) {
-      for (AtomicReader reader : completeReaderSet) {
+    for (AtomicReader reader : completeReaderSet) {
+      if (!closeSubReaders) {
         reader.incRef();
       }
+      reader.registerParentReader(this);
     }
   }
 
@@ -132,11 +151,11 @@ public final class ParallelAtomicReader 
   private final class ParallelFieldsEnum extends FieldsEnum {
     private String currentField;
     private final Iterator<String> keys;
-    private final Fields fields;
+    private final ParallelFields fields;
     
-    ParallelFieldsEnum(Fields fields) {
+    ParallelFieldsEnum(ParallelFields fields) {
       this.fields = fields;
-      keys = fieldToReader.keySet().iterator();
+      keys = fields.fields.keySet().iterator();
     }
     
     @Override
@@ -158,7 +177,7 @@ public final class ParallelAtomicReader 
   
   // Single instance of this, per ParallelReader instance
   private final class ParallelFields extends Fields {
-    final HashMap<String,Terms> fields = new HashMap<String,Terms>();
+    final Map<String,Terms> fields = new TreeMap<String,Terms>();
     
     ParallelFields() {
     }
@@ -197,11 +216,6 @@ public final class ParallelAtomicReader 
   @Override
   public Fields fields() {
     ensureOpen();
-    // we cache the inner field instances, so we must check
-    // that the delegate readers are really still open:
-    for (final AtomicReader reader : parallelReaders) {
-      reader.ensureOpen();
-    }
     return fields;
   }
   
@@ -231,15 +245,17 @@ public final class ParallelAtomicReader 
     }
   }
   
-  // get all vectors
   @Override
   public Fields getTermVectors(int docID) throws IOException {
     ensureOpen();
-    ParallelFields fields = new ParallelFields();
-    for (Map.Entry<String,AtomicReader> ent : fieldToReader.entrySet()) {
+    ParallelFields fields = null;
+    for (Map.Entry<String,AtomicReader> ent : tvFieldToReader.entrySet()) {
       String fieldName = ent.getKey();
       Terms vector = ent.getValue().getTermVector(docID, fieldName);
       if (vector != null) {
+        if (fields == null) {
+          fields = new ParallelFields();
+        }
         fields.addField(fieldName, vector);
       }
     }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/ParallelCompositeReader.java Tue Mar  6 23:17:08 2012
@@ -46,7 +46,7 @@ import java.util.Set;
  * by number of documents per segment. If you use different {@link MergePolicy}s
  * it might happen that the segment structure of your index is no longer predictable.
  */
-public final class ParallelCompositeReader extends BaseMultiReader<IndexReader> {
+public final class ParallelCompositeReader extends BaseCompositeReader<IndexReader> {
   private final boolean closeSubReaders;
   private final Set<CompositeReader> completeReaderSet =
     Collections.newSetFromMap(new IdentityHashMap<CompositeReader,Boolean>());

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/PerDocWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/PerDocWriteState.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/PerDocWriteState.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/PerDocWriteState.java Tue Mar  6 23:17:08 2012
@@ -1,22 +1,4 @@
 package org.apache.lucene.index;
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-import java.io.PrintStream;
-
 import org.apache.lucene.codecs.PerDocConsumer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SlowCompositeReaderWrapper.java Tue Mar  6 23:17:08 2012
@@ -68,6 +68,7 @@ public final class SlowCompositeReaderWr
     in = reader;
     fields = MultiFields.getFields(in);
     liveDocs = MultiFields.getLiveDocs(in);
+    in.registerParentReader(this);
   }
 
   @Override
@@ -78,7 +79,6 @@ public final class SlowCompositeReaderWr
   @Override
   public Fields fields() throws IOException {
     ensureOpen();
-    in.ensureOpen(); // as we cached the fields, we better check the original reader
     return fields;
   }
 
@@ -127,7 +127,6 @@ public final class SlowCompositeReaderWr
   @Override
   public Bits getLiveDocs() {
     ensureOpen();
-    in.ensureOpen(); // as we cached the liveDocs, we better check the original reader
     return liveDocs;
   }
 

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/SortedBytesMergeUtils.java Tue Mar  6 23:17:08 2012
@@ -81,7 +81,7 @@ public final class SortedBytesMergeUtils
     }
   }
 
-  public static List<SortedSourceSlice> buildSlices(int[] docBases ,int[][] docMaps,
+  public static List<SortedSourceSlice> buildSlices(int[] docBases, int[][] docMaps,
       DocValues[] docValues, MergeContext ctx) throws IOException {
     final List<SortedSourceSlice> slices = new ArrayList<SortedSourceSlice>();
     for (int i = 0; i < docValues.length; i++) {
@@ -111,7 +111,7 @@ public final class SortedBytesMergeUtils
    * mapping in docIDToRelativeOrd. After the merge SortedSourceSlice#ordMapping
    * contains the new global ordinals for the relative index.
    */
-  private static void createOrdMapping(int[] docBases ,int[][] docMaps,
+  private static void createOrdMapping(int[] docBases, int[][] docMaps,
       SortedSourceSlice currentSlice) {
     final int readerIdx = currentSlice.readerIdx;
     final int[] currentDocMap = docMaps[readerIdx];

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java Tue Mar  6 23:17:08 2012
@@ -23,6 +23,7 @@ import java.util.Comparator;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
 
 /** Iterator to seek ({@link #seekCeil(BytesRef)}, {@link
  * #seekExact(BytesRef,boolean)}) or step through ({@link
@@ -40,7 +41,7 @@ import org.apache.lucene.util.BytesRef;
  * of the <code>seek</code> methods.
  *
  * @lucene.experimental */
-public abstract class TermsEnum {
+public abstract class TermsEnum implements BytesRefIterator {
 
   private AttributeSource atts = null;
 
@@ -114,14 +115,6 @@ public abstract class TermsEnum {
     }
   }
 
-  /** Increments the enumeration to the next term.
-   *  Returns the resulting term, or null if the end was
-   *  hit (which means the enum is unpositioned).  The
-   *  returned BytesRef may be re-used across calls to next.
-   *  After this method returns null, do not call it again:
-   *  the results are undefined. */
-  public abstract BytesRef next() throws IOException;
-
   /** Returns current term. Do not call this when the enum
    *  is unpositioned. */
   public abstract BytesRef term() throws IOException;
@@ -186,13 +179,6 @@ public abstract class TermsEnum {
       }
     };
   }
-  
-  /** Return the {@link BytesRef} Comparator used to sort
-   *  terms provided by the iterator.  This may return
-   *  null if there are no terms.  Callers may invoke this
-   *  method many times, so it's best to cache a single
-   *  instance & reuse it. */
-  public abstract Comparator<BytesRef> getComparator() throws IOException;
 
   /** An empty TermsEnum for quickly returning an empty instance e.g.
    * in {@link org.apache.lucene.search.MultiTermQuery}

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Tue Mar  6 23:17:08 2012
@@ -203,8 +203,6 @@ final class BooleanScorer extends Scorer
   private final int minNrShouldMatch;
   private int end;
   private Bucket current;
-  private int doc = -1;
-
   // Any time a prohibited clause matches we set bit 0:
   private static final int PROHIBITED_MASK = 1;
   

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java Tue Mar  6 23:17:08 2012
@@ -25,7 +25,6 @@ import java.util.List;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.Scorer.ChildScorer;
 
 /* See the description in BooleanScorer.java, comparing
  * BooleanScorer & BooleanScorer2 */

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ConjunctionTermScorer.java Tue Mar  6 23:17:08 2012
@@ -49,7 +49,7 @@ class ConjunctionTermScorer extends Scor
 
   private int doNext(int doc) throws IOException {
     do {
-      if (lead.doc == DocsEnum.NO_MORE_DOCS) {
+      if (lead.doc == DocIdSetIterator.NO_MORE_DOCS) {
         return NO_MORE_DOCS;
       }
       advanceHead: do {

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Tue Mar  6 23:17:08 2012
@@ -76,7 +76,7 @@ final class ExactPhraseScorer extends Sc
       // freq of rarest 2 terms is close:
       final boolean useAdvance = postings[i].docFreq > 5*postings[0].docFreq;
       chunkStates[i] = new ChunkState(postings[i].postings, -postings[i].position, useAdvance);
-      if (i > 0 && postings[i].postings.nextDoc() == DocsEnum.NO_MORE_DOCS) {
+      if (i > 0 && postings[i].postings.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) {
         noDocs = true;
         return;
       }
@@ -89,7 +89,7 @@ final class ExactPhraseScorer extends Sc
 
       // first (rarest) term
       final int doc = chunkStates[0].posEnum.nextDoc();
-      if (doc == DocsEnum.NO_MORE_DOCS) {
+      if (doc == DocIdSetIterator.NO_MORE_DOCS) {
         docID = doc;
         return doc;
       }
@@ -140,8 +140,8 @@ final class ExactPhraseScorer extends Sc
 
     // first term
     int doc = chunkStates[0].posEnum.advance(target);
-    if (doc == DocsEnum.NO_MORE_DOCS) {
-      docID = DocsEnum.NO_MORE_DOCS;
+    if (doc == DocIdSetIterator.NO_MORE_DOCS) {
+      docID = DocIdSetIterator.NO_MORE_DOCS;
       return doc;
     }
 
@@ -171,7 +171,7 @@ final class ExactPhraseScorer extends Sc
       }
 
       doc = chunkStates[0].posEnum.nextDoc();
-      if (doc == DocsEnum.NO_MORE_DOCS) {
+      if (doc == DocIdSetIterator.NO_MORE_DOCS) {
         docID = doc;
         return doc;
       }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Tue Mar  6 23:17:08 2012
@@ -367,7 +367,7 @@ class FieldCacheImpl implements FieldCac
             docs = termsEnum.docs(null, docs, false);
             while (true) {
               final int docID = docs.nextDoc();
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               }
               retArray[docID] = termval;
@@ -440,7 +440,7 @@ class FieldCacheImpl implements FieldCac
             docs = termsEnum.docs(null, docs, false);
             while (true) {
               final int docID = docs.nextDoc();
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               }
               retArray[docID] = termval;
@@ -544,7 +544,7 @@ class FieldCacheImpl implements FieldCac
             docs = termsEnum.docs(null, docs, false);
             while (true) {
               final int docID = docs.nextDoc();
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               }
               retArray[docID] = termval;
@@ -612,7 +612,7 @@ class FieldCacheImpl implements FieldCac
           // TODO: use bulk API
           while (true) {
             final int docID = docs.nextDoc();
-            if (docID == DocsEnum.NO_MORE_DOCS) {
+            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }
             res.set(docID);
@@ -694,7 +694,7 @@ class FieldCacheImpl implements FieldCac
             docs = termsEnum.docs(null, docs, false);
             while (true) {
               final int docID = docs.nextDoc();
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               }
               retArray[docID] = termval;
@@ -782,7 +782,7 @@ class FieldCacheImpl implements FieldCac
             docs = termsEnum.docs(null, docs, false);
             while (true) {
               final int docID = docs.nextDoc();
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               }
               retArray[docID] = termval;
@@ -871,7 +871,7 @@ class FieldCacheImpl implements FieldCac
             docs = termsEnum.docs(null, docs, false);
             while (true) {
               final int docID = docs.nextDoc();
-              if (docID == DocsEnum.NO_MORE_DOCS) {
+              if (docID == DocIdSetIterator.NO_MORE_DOCS) {
                 break;
               }
               retArray[docID] = termval;
@@ -1052,7 +1052,7 @@ class FieldCacheImpl implements FieldCac
       }
 
       @Override
-      public Comparator<BytesRef> getComparator() throws IOException {
+      public Comparator<BytesRef> getComparator() {
         return BytesRef.getUTF8SortedAsUnicodeComparator();
       }
 
@@ -1172,7 +1172,7 @@ class FieldCacheImpl implements FieldCac
           docs = termsEnum.docs(null, docs, false);
           while (true) {
             final int docID = docs.nextDoc();
-            if (docID == DocsEnum.NO_MORE_DOCS) {
+            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }
             docToTermOrd.set(docID, termOrd);
@@ -1293,7 +1293,7 @@ class FieldCacheImpl implements FieldCac
           docs = termsEnum.docs(null, docs, false);
           while (true) {
             final int docID = docs.nextDoc();
-            if (docID == DocsEnum.NO_MORE_DOCS) {
+            if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }
             docToOffset.set(docID, pointer);

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

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Tue Mar  6 23:17:08 2012
@@ -150,7 +150,7 @@ public abstract class FieldComparator<T>
    *   comparator across segments
    * @throws IOException
    */
-  public abstract FieldComparator setNextReader(AtomicReaderContext context) throws IOException;
+  public abstract FieldComparator<T> setNextReader(AtomicReaderContext context) throws IOException;
 
   /** Sets the Scorer to use in case a document's score is
    *  needed.
@@ -201,7 +201,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<T> setNextReader(AtomicReaderContext context) throws IOException {
       if (missingValue != null) {
         docsWithField = FieldCache.DEFAULT.getDocsWithField(context.reader(), field);
         // optimization to remove unneeded checks on the bit interface:
@@ -258,7 +258,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Byte> setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
       currentReaderValues = FieldCache.DEFAULT.getBytes(context.reader(), field, parser, missingValue != null);
@@ -335,7 +335,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Double> setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
       currentReaderValues = FieldCache.DEFAULT.getDoubles(context.reader(), field, parser, missingValue != null);
@@ -396,7 +396,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Double> setNextReader(AtomicReaderContext context) throws IOException {
       final DocValues docValues = context.reader().docValues(field);
       if (docValues != null) {
         currentReaderValues = docValues.getSource(); 
@@ -478,7 +478,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Float> setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
       currentReaderValues = FieldCache.DEFAULT.getFloats(context.reader(), field, parser, missingValue != null);
@@ -540,7 +540,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Short> setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
       currentReaderValues = FieldCache.DEFAULT.getShorts(context.reader(), field, parser, missingValue != null);
@@ -624,7 +624,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Integer> setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
       currentReaderValues = FieldCache.DEFAULT.getInts(context.reader(), field, parser, missingValue != null);
@@ -689,7 +689,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Long> setNextReader(AtomicReaderContext context) throws IOException {
       DocValues docValues = context.reader().docValues(field);
       if (docValues != null) {
         currentReaderValues = docValues.getSource();
@@ -772,7 +772,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<Long> setNextReader(AtomicReaderContext context) throws IOException {
       // NOTE: must do this before calling super otherwise
       // we compute the docsWithField Bits twice!
       currentReaderValues = FieldCache.DEFAULT.getLongs(context.reader(), field, parser, missingValue != null);
@@ -824,7 +824,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) {
+    public FieldComparator<Float> setNextReader(AtomicReaderContext context) {
       return this;
     }
     
@@ -887,7 +887,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) {
+    public FieldComparator<Integer> setNextReader(AtomicReaderContext context) {
       // TODO: can we "map" our docIDs to the current
       // reader? saves having to then subtract on every
       // compare call
@@ -1007,7 +1007,7 @@ public abstract class FieldComparator<T>
     abstract class PerSegmentComparator extends FieldComparator<BytesRef> {
       
       @Override
-      public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
         return TermOrdValComparator.this.setNextReader(context);
       }
 
@@ -1055,32 +1055,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = (readerOrds[doc]&0xFF);
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - (readerOrds[doc]&0xFF);
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = readerOrds[doc]&0xFF;
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-
-          if (bottomValue == null) {
-            if (order == 0) {
-              // unset
-              return 0;
-            }
-            // bottom wins
-            return -1;
-          } else if (order == 0) {
-            // doc wins
-            return 1;
-          }
-          termsIndex.lookup(order, tempBR);
-          return bottomValue.compareTo(tempBR);
+          return -1;
         }
       }
 
@@ -1116,32 +1101,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = (readerOrds[doc]&0xFFFF);
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - (readerOrds[doc]&0xFFFF);
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = readerOrds[doc]&0xFFFF;
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-
-          if (bottomValue == null) {
-            if (order == 0) {
-              // unset
-              return 0;
-            }
-            // bottom wins
-            return -1;
-          } else if (order == 0) {
-            // doc wins
-            return 1;
-          }
-          termsIndex.lookup(order, tempBR);
-          return bottomValue.compareTo(tempBR);
+          return -1;
         }
       }
 
@@ -1177,32 +1147,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = readerOrds[doc];
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - readerOrds[doc];
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = readerOrds[doc];
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-
-          if (bottomValue == null) {
-            if (order == 0) {
-              // unset
-              return 0;
-            }
-            // bottom wins
-            return -1;
-          } else if (order == 0) {
-            // doc wins
-            return 1;
-          }
-          termsIndex.lookup(order, tempBR);
-          return bottomValue.compareTo(tempBR);
+          return -1;
         }
       }
 
@@ -1239,32 +1194,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = (int) readerOrds.get(doc);
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - (int) readerOrds.get(doc);
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = (int) readerOrds.get(doc);
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-
-          if (bottomValue == null) {
-            if (order == 0) {
-              // unset
-              return 0;
-            }
-            // bottom wins
-            return -1;
-          } else if (order == 0) {
-            // doc wins
-            return 1;
-          }
-          termsIndex.lookup(order, tempBR);
-          return bottomValue.compareTo(tempBR);
+          return -1;
         }
       }
 
@@ -1286,11 +1226,11 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
       final int docBase = context.docBase;
       termsIndex = FieldCache.DEFAULT.getTermsIndex(context.reader(), field);
       final PackedInts.Reader docToOrd = termsIndex.getDocToOrd();
-      FieldComparator perSegComp = null;
+      FieldComparator<BytesRef> perSegComp = null;
       if (docToOrd.hasArray()) {
         final Object arr = docToOrd.getArray();
         if (arr instanceof byte[]) {
@@ -1457,7 +1397,7 @@ public abstract class FieldComparator<T>
     abstract class PerSegmentComparator extends FieldComparator<BytesRef> {
       
       @Override
-      public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+      public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
         return TermOrdValDocValuesComparator.this.setNextReader(context);
       }
 
@@ -1499,21 +1439,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = readerOrds[doc]&0xFF;
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - (readerOrds[doc]&0xFF);
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = readerOrds[doc]&0xFF;
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-
-          termsIndex.getByOrd(order, tempBR);
-          return comp.compare(bottomValue, tempBR);
+          return -1;
         }
       }
 
@@ -1544,21 +1480,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = readerOrds[doc]&0xFFFF;
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - (readerOrds[doc]&0xFFFF);
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = readerOrds[doc]&0xFFFF;
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-
-          termsIndex.getByOrd(order, tempBR);
-          return comp.compare(bottomValue, tempBR);
+          return -1;
         }
       }
 
@@ -1589,20 +1521,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = readerOrds[doc];
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - readerOrds[doc];
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = readerOrds[doc];
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-          termsIndex.getByOrd(order, tempBR);
-          return comp.compare(bottomValue, tempBR);
+          return -1;
         }
       }
 
@@ -1632,20 +1561,17 @@ public abstract class FieldComparator<T>
       @Override
       public int compareBottom(int doc) {
         assert bottomSlot != -1;
+        final int docOrd = (int) readerOrds.get(doc);
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - (int) readerOrds.get(doc);
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = (int) readerOrds.get(doc);
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-          termsIndex.getByOrd(order, tempBR);
-          return comp.compare(bottomValue, tempBR);
+          return -1;
         }
       }
 
@@ -1672,21 +1598,17 @@ public abstract class FieldComparator<T>
 
       @Override
       public int compareBottom(int doc) {
-        assert bottomSlot != -1;
+        final int docOrd = termsIndex.ord(doc);
         if (bottomSameReader) {
           // ord is precisely comparable, even in the equal case
-          return bottomOrd - termsIndex.ord(doc);
+          return bottomOrd - docOrd;
+        } else if (bottomOrd >= docOrd) {
+          // the equals case always means bottom is > doc
+          // (because we set bottomOrd to the lower bound in
+          // setBottom):
+          return 1;
         } else {
-          // ord is only approx comparable: if they are not
-          // equal, we can use that; if they are equal, we
-          // must fallback to compare by value
-          final int order = termsIndex.ord(doc);
-          final int cmp = bottomOrd - order;
-          if (cmp != 0) {
-            return cmp;
-          }
-          termsIndex.getByOrd(order, tempBR);
-          return comp.compare(bottomValue, tempBR);
+          return -1;
         }
       }
 
@@ -1703,7 +1625,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
       final int docBase = context.docBase;
 
       final DocValues dv = context.reader().docValues(field);
@@ -1724,7 +1646,7 @@ public abstract class FieldComparator<T>
 
       comp = termsIndex.getComparator();
 
-      FieldComparator perSegComp = null;
+      FieldComparator<BytesRef> perSegComp = null;
       if (termsIndex.hasPackedDocToOrd()) {
         final PackedInts.Reader docToOrd = termsIndex.getDocToOrd();
         if (docToOrd.hasArray()) {
@@ -1775,7 +1697,7 @@ public abstract class FieldComparator<T>
           bottomSameReader = true;
           readerGen[bottomSlot] = currentReaderGen;
         } else {
-          final int index = termsIndex.getByValue(bottomValue, tempBR);
+          final int index = termsIndex.getOrdByValue(bottomValue, tempBR);
           if (index < 0) {
             bottomOrd = -index - 2;
             bottomSameReader = false;
@@ -1852,7 +1774,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
       docTerms = FieldCache.DEFAULT.getTerms(context.reader(), field);
       return this;
     }
@@ -1921,7 +1843,7 @@ public abstract class FieldComparator<T>
     }
 
     @Override
-    public FieldComparator setNextReader(AtomicReaderContext context) throws IOException {
+    public FieldComparator<BytesRef> setNextReader(AtomicReaderContext context) throws IOException {
       final DocValues dv = context.reader().docValues(field);
       if (dv != null) {
         docTerms = dv.getSource();

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparatorSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparatorSource.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparatorSource.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldComparatorSource.java Tue Mar  6 23:17:08 2012
@@ -36,6 +36,6 @@ public abstract class FieldComparatorSou
    * @throws IOException
    *           If an error occurs reading the index.
    */
-  public abstract FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed)
+  public abstract FieldComparator<?> newComparator(String fieldname, int numHits, int sortPos, boolean reversed)
       throws IOException;
 }

Modified: lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=1297785&r1=1297784&r2=1297785&view=diff
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/lucene/core/src/java/org/apache/lucene/search/FieldValueHitQueue.java Tue Mar  6 23:17:08 2012
@@ -129,6 +129,7 @@ public abstract class FieldValueHitQueue
   }
   
   // prevent instantiation and extension.
+  @SuppressWarnings({"rawtypes","unchecked"})
   private FieldValueHitQueue(SortField[] fields, int size) {
     super(size);
     // When we get here, fields.length is guaranteed to be > 0, therefore no
@@ -169,7 +170,7 @@ public abstract class FieldValueHitQueue
     }
   }
   
-  public FieldComparator[] getComparators() {
+  public FieldComparator<?>[] getComparators() {
     return comparators;
   }
 
@@ -177,15 +178,15 @@ public abstract class FieldValueHitQueue
     return reverseMul;
   }
 
-  public void setComparator(int pos, FieldComparator comparator) {
+  public void setComparator(int pos, FieldComparator<?> comparator) {
     if (pos==0) firstComparator = comparator;
     comparators[pos] = comparator;
   }
 
   /** Stores the sort criteria being used. */
   protected final SortField[] fields;
-  protected final FieldComparator[] comparators;  // use setComparator to change this array
-  protected FieldComparator firstComparator;      // this must always be equal to comparators[0]
+  protected final FieldComparator<?>[] comparators;  // use setComparator to change this array
+  protected FieldComparator<?> firstComparator;      // this must always be equal to comparators[0]
   protected final int[] reverseMul;
 
   @Override