lucene-java-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From markrmil...@apache.org
Subject svn commit: r880963 [6/7] - in /lucene/java/branches/flex_1458: ./ contrib/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/ar/ contrib/analyzers/common/src/java/org/apache/lucene/analysis/br/ contrib/analyzers/common/src/java/org/apache/l...
Date Mon, 16 Nov 2009 21:24:47 GMT
Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldConsumers.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldConsumers.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldConsumers.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocFieldConsumers.java Mon Nov 16 21:24:41 2009
@@ -29,8 +29,6 @@
 /** This is just a "splitter" class: it lets you wrap two
  *  DocFieldConsumer instances as a single consumer. */
 
-// TODO: Fix the unchecked collections, I do not understand the whole code here -- Uwe
-@SuppressWarnings("unchecked")
 final class DocFieldConsumers extends DocFieldConsumer {
   final DocFieldConsumer one;
   final DocFieldConsumer two;
@@ -50,21 +48,18 @@
   @Override
   public void flush(Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
 
-    Map oneThreadsAndFields = new HashMap();
-    Map twoThreadsAndFields = new HashMap();
+    Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> oneThreadsAndFields = new HashMap<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>>();
+    Map<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> twoThreadsAndFields = new HashMap<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>>();
 
-    Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
+    for (Map.Entry<DocFieldConsumerPerThread,Collection<DocFieldConsumerPerField>> entry : threadsAndFields.entrySet()) {
 
-      Map.Entry entry = (Map.Entry) it.next();
+      final DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
 
-      DocFieldConsumersPerThread perThread = (DocFieldConsumersPerThread) entry.getKey();
+      final Collection<DocFieldConsumerPerField> fields = entry.getValue();
 
-      Collection fields = (Collection) entry.getValue();
-
-      Iterator fieldsIt = fields.iterator();
-      Collection oneFields = new HashSet();
-      Collection twoFields = new HashSet();
+      Iterator<DocFieldConsumerPerField> fieldsIt = fields.iterator();
+      Collection<DocFieldConsumerPerField> oneFields = new HashSet<DocFieldConsumerPerField>();
+      Collection<DocFieldConsumerPerField> twoFields = new HashSet<DocFieldConsumerPerField>();
       while(fieldsIt.hasNext()) {
         DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldsIt.next();
         oneFields.add(perField.one);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/DocumentsWriter.java Mon Nov 16 21:24:41 2009
@@ -39,6 +39,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /**
  * This class accepts multiple added documents and directly
@@ -514,10 +515,7 @@
       try {
         wait();
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
     }
 
@@ -862,10 +860,7 @@
       try {
         wait();
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
     }
 
@@ -1121,10 +1116,7 @@
       try {
         wait();
       } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
+        throw new ThreadInterruptedException(ie);
       }
     } while (!waitQueue.doResume());
   }
@@ -1209,7 +1201,7 @@
           numBytesAlloc += BYTE_BLOCK_SIZE;
           b = new byte[BYTE_BLOCK_SIZE];
         } else
-          b = (byte[]) freeByteBlocks.remove(size-1);
+          b = freeByteBlocks.remove(size-1);
         if (trackAllocations)
           numBytesUsed += BYTE_BLOCK_SIZE;
         assert numBytesUsed <= numBytesAlloc;
@@ -1249,7 +1241,7 @@
       numBytesAlloc += INT_BLOCK_SIZE*INT_NUM_BYTE;
       b = new int[INT_BLOCK_SIZE];
     } else
-      b = (int[]) freeIntBlocks.remove(size-1);
+      b = freeIntBlocks.remove(size-1);
     if (trackAllocations)
       numBytesUsed += INT_BLOCK_SIZE*INT_NUM_BYTE;
     assert numBytesUsed <= numBytesAlloc;
@@ -1292,7 +1284,7 @@
       numBytesAlloc += CHAR_BLOCK_SIZE * CHAR_NUM_BYTE;
       c = new char[CHAR_BLOCK_SIZE];
     } else
-      c = (char[]) freeCharBlocks.remove(size-1);
+      c = freeCharBlocks.remove(size-1);
     // We always track allocations of char blocks, for now,
     // because nothing that skips allocation tracking
     // (currently only term vectors) uses its own char

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FieldInfos.java Mon Nov 16 21:24:41 2009
@@ -288,7 +288,7 @@
    * doesn't exist.
    */  
   public FieldInfo fieldInfo(int fieldNumber) {
-	return (fieldNumber >= 0) ? (FieldInfo) byNumber.get(fieldNumber) : null;
+	return (fieldNumber >= 0) ? byNumber.get(fieldNumber) : null;
   }
 
   public int size() {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Mon Nov 16 21:24:41 2009
@@ -31,8 +31,6 @@
 import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.util.UnicodeUtil;
 
-// TODO: Fix the unchecked collections, I do not understand the whole code here -- Uwe
-@SuppressWarnings("unchecked")
 final class FreqProxTermsWriter extends TermsHashConsumer {
 
   @Override
@@ -80,18 +78,18 @@
 
     // Gather all FieldData's that have postings, across all
     // ThreadStates
-    List allFields = new ArrayList();
+    List<FreqProxTermsWriterPerField> allFields = new ArrayList<FreqProxTermsWriterPerField>();
     
     flushedDocCount = state.numDocs;
 
-    Iterator it = threadsAndFields.entrySet().iterator();
+    Iterator<Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>> it = threadsAndFields.entrySet().iterator();
     while(it.hasNext()) {
 
-      Map.Entry entry = (Map.Entry) it.next();
+      Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry = it.next();
 
-      Collection fields = (Collection) entry.getValue();
+      Collection<TermsHashConsumerPerField> fields = entry.getValue();
 
-      Iterator fieldsIt = fields.iterator();
+      Iterator<TermsHashConsumerPerField> fieldsIt = fields.iterator();
 
       while(fieldsIt.hasNext()) {
         FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) fieldsIt.next();
@@ -122,16 +120,16 @@
 
     int start = 0;
     while(start < numAllFields) {
-      final FieldInfo fieldInfo = ((FreqProxTermsWriterPerField) allFields.get(start)).fieldInfo;
+      final FieldInfo fieldInfo = allFields.get(start).fieldInfo;
       final String fieldName = fieldInfo.name;
 
       int end = start+1;
-      while(end < numAllFields && ((FreqProxTermsWriterPerField) allFields.get(end)).fieldInfo.name.equals(fieldName))
+      while(end < numAllFields && allFields.get(end).fieldInfo.name.equals(fieldName))
         end++;
       
       FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start];
       for(int i=start;i<end;i++) {
-        fields[i-start] = (FreqProxTermsWriterPerField) allFields.get(i);
+        fields[i-start] = allFields.get(i);
 
         // Aggregate the storePayload as seen by the same
         // field across multiple threads
@@ -153,9 +151,7 @@
       start = end;
     }
 
-    it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
-      Map.Entry entry = (Map.Entry) it.next();
+    for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
       FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey();
       perThread.termsHashPerThread.reset(true);
     }
@@ -302,6 +298,7 @@
 
   final UnicodeUtil.UTF8Result termsUTF8 = new UnicodeUtil.UTF8Result();
 
+  //nocommit: needed?
   void files(Collection<String> files) {}
 
   static final class PostingList extends RawPostingList {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Mon Nov 16 21:24:41 2009
@@ -24,7 +24,7 @@
 // TODO: break into separate freq and prox writers as
 // codecs; make separate container (tii/tis/skip/*) that can
 // be configured as any number of files 1..N
-final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable {
+final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable<FreqProxTermsWriterPerField> {
 
   final FreqProxTermsWriterPerThread perThread;
   final TermsHashPerField termsHashPerField;
@@ -59,8 +59,7 @@
   @Override
   void skippingLongTerm() throws IOException {}
 
-  public int compareTo(Object other0) {
-    FreqProxTermsWriterPerField other = (FreqProxTermsWriterPerField) other0;
+  public int compareTo(FreqProxTermsWriterPerField other) {
     return fieldInfo.name.compareTo(other.fieldInfo.name);
   }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexDeletionPolicy.java Mon Nov 16 21:24:41 2009
@@ -84,7 +84,7 @@
    * <p>This method is only called when {@link
    * IndexWriter#commit} or {@link IndexWriter#close} is
    * called, or possibly not at all if the {@link
-   * IndexWriter#abort} is called.
+   * IndexWriter#rollback} is called.
    *
    * <p><u>Note:</u> the last CommitPoint is the most recent one,
    * i.e. the "front index state". Be careful not to delete it,

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexFileDeleter.java Mon Nov 16 21:24:41 2009
@@ -583,7 +583,7 @@
    * equals.
    */
 
-  final private static class CommitPoint extends IndexCommit implements Comparable {
+  final private static class CommitPoint extends IndexCommit implements Comparable<CommitPoint> {
 
     long gen;
     Collection<String> files;
@@ -662,8 +662,7 @@
       return deleted;
     }
 
-    public int compareTo(Object obj) {
-      CommitPoint commit = (CommitPoint) obj;
+    public int compareTo(CommitPoint commit) {
       if (gen < commit.gen) {
         return -1;
       } else if (gen > commit.gen) {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexReader.java Mon Nov 16 21:24:41 2009
@@ -546,7 +546,27 @@
   }
 
   /**
-   * Version number when this IndexReader was opened. Not implemented in the IndexReader base class.
+   * Version number when this IndexReader was opened. Not
+   * implemented in the IndexReader base class.
+   *
+   * <p>If this reader is based on a Directory (ie, was
+   * created by calling {@link #open}, or {@link #reopen} on
+   * a reader based on a Directory), then this method
+   * returns the version recorded in the commit that the
+   * reader opened.  This version is advanced every time
+   * {@link IndexWriter#commit} is called.</p>
+   *
+   * <p>If instead this reader is a near real-time reader
+   * (ie, obtained by a call to {@link
+   * IndexWriter#getReader}, or by calling {@link #reopen}
+   * on a near real-time reader), then this method returns
+   * the version of the last commit done by the writer.
+   * Note that even as further changes are made with the
+   * writer, the version will not changed until a commit is
+   * completed.  Thus, you should not rely on this method to
+   * determine when a near real-time reader should be
+   * opened.  Use {@link #isCurrent} instead.</p>
+   *
    * @throws UnsupportedOperationException unless overridden in subclass
    */
   public long getVersion() {
@@ -566,18 +586,28 @@
   }
 
   /**
-   * Check whether this IndexReader is still using the
-   * current (i.e., most recently committed) version of the
-   * index.  If a writer has committed any changes to the
-   * index since this reader was opened, this will return
-   * <code>false</code>, in which case you must open a new
-   * IndexReader in order to see the changes.  Changes must
-   * be committed using  {@link IndexWriter#commit} to be
-   * visible to readers.
-   * 
-   * <p>
-   * Not implemented in the IndexReader base class.
-   * </p>
+   * Check whether any new changes have occurred to the
+   * index since this reader was opened.
+   *
+   * <p>If this reader is based on a Directory (ie, was
+   * created by calling {@link #open}, or {@link #reopen} on
+   * a reader based on a Directory), then this method checks
+   * if any further commits (see {@link IndexWriter#commit}
+   * have occurred in that directory).</p>
+   *
+   * <p>If instead this reader is a near real-time reader
+   * (ie, obtained by a call to {@link
+   * IndexWriter#getReader}, or by calling {@link #reopen}
+   * on a near real-time reader), then this method checks if
+   * either a new commmit has occurred, or any new
+   * uncommitted changes have taken place via the writer.
+   * Note that even if the writer has only performed
+   * merging, this method will still return false.</p>
+   *
+   * <p>In any event, if this returns false, you should call
+   * {@link #reopen} to get a new reader that sees the
+   * changes.</p>
+   *
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    * @throws UnsupportedOperationException unless overridden in subclass
@@ -1254,7 +1284,7 @@
    *  one commit point.  But if you're using a custom {@link
    *  IndexDeletionPolicy} then there could be many commits.
    *  Once you have a given commit, you can open a reader on
-   *  it by calling {@link IndexReader#open(IndexCommit)}
+   *  it by calling {@link IndexReader#open(IndexCommit,boolean)}
    *  There must be at least one commit in
    *  the Directory, else this method throws {@link
    *  java.io.IOException}.  Note that if a commit is in

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/IndexWriter.java Mon Nov 16 21:24:41 2009
@@ -29,6 +29,7 @@
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.util.Constants;
 import org.apache.lucene.index.codecs.Codecs;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 import java.io.IOException;
 import java.io.Closeable;
@@ -47,13 +48,13 @@
   An <code>IndexWriter</code> creates and maintains an index.
 
   <p>The <code>create</code> argument to the {@link
-  #IndexWriter(Directory, Analyzer, boolean) constructor} determines 
+  #IndexWriter(Directory, Analyzer, boolean, MaxFieldLength) constructor} determines 
   whether a new index is created, or whether an existing index is
   opened.  Note that you can open an index with <code>create=true</code>
   even while readers are using the index.  The old readers will 
   continue to search the "point in time" snapshot they had opened, 
   and won't see the newly created index until they re-open.  There are
-  also {@link #IndexWriter(Directory, Analyzer) constructors}
+  also {@link #IndexWriter(Directory, Analyzer, MaxFieldLength) constructors}
   with no <code>create</code> argument which will create a new index
   if there is not already an index at the provided path and otherwise 
   open the existing index.</p>
@@ -145,6 +146,13 @@
   synchronize on the <code>IndexWriter</code> instance as
   this may cause deadlock; use your own (non-Lucene) objects
   instead. </p>
+  
+  <p><b>NOTE</b>: If you call
+  <code>Thread.interrupt()</code> on a thread that's within
+  IndexWriter, IndexWriter will try to catch this (eg, if
+  it's in a wait() or Thread.sleep()), and will then throw
+  the unchecked exception {@link ThreadInterruptedException}
+  and <b>clear</b> the interrupt status on the thread.</p>
 */
 
 /*
@@ -4532,10 +4540,7 @@
             try {
               synced.wait();
             } catch (InterruptedException ie) {
-              // In 3.0 we will change this to throw
-              // InterruptedException instead
-              Thread.currentThread().interrupt();
-              throw new RuntimeException(ie);
+              throw new ThreadInterruptedException(ie);
             }
         }
       }
@@ -4553,10 +4558,7 @@
     try {
       wait(1000);
     } catch (InterruptedException ie) {
-      // In 3.0 we will change this to throw
-      // InterruptedException instead
-      Thread.currentThread().interrupt();
-      throw new RuntimeException(ie);
+      throw new ThreadInterruptedException(ie);
     }
   }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/KeepOnlyLastCommitDeletionPolicy.java Mon Nov 16 21:24:41 2009
@@ -44,7 +44,7 @@
     // called by onInit above):
     int size = commits.size();
     for(int i=0;i<size-1;i++) {
-      ((IndexCommit) commits.get(i)).delete();
+      commits.get(i).delete();
     }
   }
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/MultiReader.java Mon Nov 16 21:24:41 2009
@@ -52,7 +52,6 @@
   * left to the subreaders. </p>
   * <p>Note that all subreaders are closed if this Multireader is closed.</p>
   * @param subReaders set of (sub)readers
- * @throws IOException 
   * @throws IOException
   */
   public MultiReader(IndexReader... subReaders) throws IOException {
@@ -410,11 +409,6 @@
     return new MultiTermPositions(this, subReaders, starts);
   }
 
-  /** @deprecated */
-  protected void doCommit() throws IOException {
-    doCommit(null);
-  }
-  
   @Override
   protected void doCommit(Map<String,String> commitUserData) throws IOException {
     for (int i = 0; i < subReaders.length; i++)

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriter.java Mon Nov 16 21:24:41 2009
@@ -62,27 +62,24 @@
   @Override
   public void flush(Map<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> threadsAndFields, SegmentWriteState state) throws IOException {
 
-    final Map byField = new HashMap();
+    final Map<FieldInfo,List<NormsWriterPerField>> byField = new HashMap<FieldInfo,List<NormsWriterPerField>>();
 
     // Typically, each thread will have encountered the same
     // field.  So first we collate by field, ie, all
     // per-thread field instances that correspond to the
     // same FieldInfo
-    final Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
-      Map.Entry entry = (Map.Entry) it.next();
+    for (final Map.Entry<InvertedDocEndConsumerPerThread,Collection<InvertedDocEndConsumerPerField>> entry : threadsAndFields.entrySet()) {
+      final Collection<InvertedDocEndConsumerPerField> fields = entry.getValue();
+      final Iterator<InvertedDocEndConsumerPerField> fieldsIt = fields.iterator();
 
-      Collection fields = (Collection) entry.getValue();
-      Iterator fieldsIt = fields.iterator();
-
-      while(fieldsIt.hasNext()) {
-        NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next();
+      while (fieldsIt.hasNext()) {
+        final NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next();
 
         if (perField.upto > 0) {
           // It has some norms
-          List l = (List) byField.get(perField.fieldInfo);
+          List<NormsWriterPerField> l = byField.get(perField.fieldInfo);
           if (l == null) {
-            l = new ArrayList();
+            l = new ArrayList<NormsWriterPerField>();
             byField.put(perField.fieldInfo, l);
           }
           l.add(perField);
@@ -108,7 +105,7 @@
 
         final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
 
-        List toMerge = (List) byField.get(fieldInfo);
+        List<NormsWriterPerField> toMerge = byField.get(fieldInfo);
         int upto = 0;
         if (toMerge != null) {
 
@@ -120,7 +117,7 @@
           int[] uptos = new int[numFields];
 
           for(int j=0;j<numFields;j++)
-            fields[j] = (NormsWriterPerField) toMerge.get(j);
+            fields[j] = toMerge.get(j);
 
           int numLeft = numFields;
               

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriterPerField.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriterPerField.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/NormsWriterPerField.java Mon Nov 16 21:24:41 2009
@@ -25,7 +25,7 @@
  *  just look at the length for the field (docState.length)
  *  and record the norm. */
 
-final class NormsWriterPerField extends InvertedDocEndConsumerPerField implements Comparable {
+final class NormsWriterPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsWriterPerField> {
 
   final NormsWriterPerThread perThread;
   final FieldInfo fieldInfo;
@@ -57,8 +57,8 @@
     upto = 0;
   }
 
-  public int compareTo(Object other) {
-    return fieldInfo.name.compareTo(((NormsWriterPerField) other).fieldInfo.name);
+  public int compareTo(NormsWriterPerField other) {
+    return fieldInfo.name.compareTo(other.fieldInfo.name);
   }
   
   @Override

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfo.java Mon Nov 16 21:24:41 2009
@@ -254,7 +254,7 @@
       delCount = -1;
       hasProx = true;
       codecName = "PreFlex";
-      diagnostics = Collections.EMPTY_MAP;
+      diagnostics = Collections.<String,String>emptyMap();
     }
     codec = codecs.lookup(codecName);
   }
@@ -722,7 +722,7 @@
     }
     return files;
   }
-  
+
   /* Called whenever any change is made that affects which
    * files this segment has. */
   private void clearFiles() {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentInfos.java Mon Nov 16 21:24:41 2009
@@ -24,6 +24,7 @@
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.NoSuchDirectoryException;
 import org.apache.lucene.index.codecs.Codecs;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -413,36 +414,14 @@
   public static long readCurrentVersion(Directory directory, final Codecs codecs)
     throws CorruptIndexException, IOException {
 
-    return ((Long) new FindSegmentsFile(directory) {
-        @Override
-        protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
-
-          IndexInput input = directory.openInput(segmentFileName);
-
-          int format = 0;
-          long version = 0;
-          try {
-            format = input.readInt();
-            if(format < 0){
-              if (format < CURRENT_FORMAT)
-                throw new CorruptIndexException("Unknown format version: " + format);
-              version = input.readLong(); // read version
-            }
-          }
-          finally {
-            input.close();
-          }
-     
-          if(format < 0)
-            return Long.valueOf(version);
-
-          // We cannot be sure about the format of the file.
-          // Therefore we have to read the whole file and cannot simply seek to the version entry.
-          SegmentInfos sis = new SegmentInfos();
-          sis.read(directory, segmentFileName, codecs);
-          return Long.valueOf(sis.getVersion());
-        }
-      }.run()).longValue();
+    // Fully read the segments file: this ensures that it's
+    // completely written so that if
+    // IndexWriter.prepareCommit has been called (but not
+    // yet commit), then the reader will still see itself as
+    // current:
+    SegmentInfos sis = new SegmentInfos();
+    sis.read(directory);
+    return sis.version;
   }
 
   /**
@@ -643,10 +622,7 @@
             try {
               Thread.sleep(defaultGenFileRetryPauseMsec);
             } catch (InterruptedException ie) {
-              // In 3.0 we will change this to throw
-              // InterruptedException instead
-              Thread.currentThread().interrupt();
-              throw new RuntimeException(ie);
+              throw new ThreadInterruptedException(ie);
             }
           }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMergeQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMergeQueue.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMergeQueue.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentMergeQueue.java Mon Nov 16 21:24:41 2009
@@ -32,4 +32,11 @@
     else
       return comparison < 0;
   }
+
+  //nocommit: still needed?
+//  final void close() throws IOException {
+//    while (top() != null)
+//      pop().close();
+//  }
+
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/SegmentReader.java Mon Nov 16 21:24:41 2009
@@ -507,7 +507,7 @@
   }
 
   Map<String,Norm> norms = new HashMap<String,Norm>();
-
+  
   /**
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Term.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Term.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Term.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/Term.java Mon Nov 16 21:24:41 2009
@@ -27,7 +27,7 @@
   Note that terms may represent more than words from text fields, but also
   things like dates, email addresses, urls, etc.  */
 
-public final class Term implements Comparable, java.io.Serializable {
+public final class Term implements Comparable<Term>, java.io.Serializable {
   String field;
   String text;
 
@@ -75,7 +75,6 @@
       return new Term(field,text,false);
   }
 
-  //@Override
   @Override
   public boolean equals(Object obj) {
     if (this == obj)
@@ -98,7 +97,6 @@
     return true;
   }
 
-  //@Override
   @Override
   public int hashCode() {
     final int prime = 31;
@@ -108,10 +106,6 @@
     return result;
   }
 
-  public int compareTo(Object other) {
-    return compareTo((Term)other);
-  }
-
   /** Compares two terms, returning a negative integer if this
     term belongs before the argument, zero if this term is equal to the
     argument, and a positive integer if this term belongs after the argument.

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHash.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/TermsHash.java Mon Nov 16 21:24:41 2009
@@ -35,9 +35,6 @@
  *  TermVectorsTermsWriter}, write their own byte streams
  *  under each term.
  */
-
-// TODO: Fix the unchecked collections, I do not understand the whole code here -- Uwe
-@SuppressWarnings("unchecked")
 final class TermsHash extends InvertedDocConsumer {
 
   final TermsHashConsumer consumer;
@@ -46,6 +43,7 @@
   final int postingsFreeChunk;
   final DocumentsWriter docWriter;
   
+  //nocommit: did I lose code for this below in merge?
   private TermsHash primaryTermsHash;
 
   private RawPostingList[] postingsFreeList = new RawPostingList[1];
@@ -90,7 +88,7 @@
       nextTermsHash.abort();
   }
 
-  void shrinkFreePostings(Map threadsAndFields, SegmentWriteState state) {
+  void shrinkFreePostings(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, SegmentWriteState state) {
 
     assert postingsFreeCount == postingsAllocCount: Thread.currentThread().getName() + ": postingsFreeCount=" + postingsFreeCount + " postingsAllocCount=" + postingsAllocCount + " consumer=" + consumer;
 
@@ -111,29 +109,26 @@
 
   @Override
   synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
-    Map childThreadsAndFields = new HashMap();
-    Map nextThreadsAndFields;
+    Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
+    Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;
 
     if (nextTermsHash != null)
-      nextThreadsAndFields = new HashMap();
+      nextThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
     else
       nextThreadsAndFields = null;
 
-    Iterator it = threadsAndFields.entrySet().iterator();
-    while(it.hasNext()) {
-
-      Map.Entry entry = (Map.Entry) it.next();
+    for (final Map.Entry<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> entry : threadsAndFields.entrySet()) {
 
       TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
 
-      Collection fields = (Collection) entry.getValue();
+      Collection<InvertedDocConsumerPerField> fields = entry.getValue();
 
-      Iterator fieldsIt = fields.iterator();
-      Collection childFields = new HashSet();
-      Collection nextChildFields;
+      Iterator<InvertedDocConsumerPerField> fieldsIt = fields.iterator();
+      Collection<TermsHashConsumerPerField> childFields = new HashSet<TermsHashConsumerPerField>();
+      Collection<InvertedDocConsumerPerField> nextChildFields;
 
       if (nextTermsHash != null)
-        nextChildFields = new HashSet();
+        nextChildFields = new HashSet<InvertedDocConsumerPerField>();
       else
         nextChildFields = null;
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/SimpleStandardTermsIndexReader.java Mon Nov 16 21:24:41 2009
@@ -69,9 +69,18 @@
 
   public SimpleStandardTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor)
     throws IOException {
-
-    IndexInput in = dir.openInput(IndexFileNames.segmentFileName(segment, StandardCodec.TERMS_INDEX_EXTENSION));
-
+    String file = IndexFileNames.segmentFileName(segment, StandardCodec.TERMS_INDEX_EXTENSION);
+    //nocommit
+    if(!dir.fileExists(file)) {
+      indexInterval = 0;
+      totalIndexInterval = 0;
+      this.indexDivisor = indexDivisor;
+      in = null;
+ 
+      return;
+    }
+    IndexInput in = dir.openInput(file);
+    
     boolean success = false;
 
     try {
@@ -125,7 +134,7 @@
         }
       } else {
         this.in = in;
-        // nocommit -- we shoudl close if index gets read on demand?
+        // nocommit -- we should close if index gets read on demand?
       }
     }
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardDocsReader.java Mon Nov 16 21:24:41 2009
@@ -40,7 +40,7 @@
 
 public class StandardDocsReader extends StandardDocsProducer {
 
-  final IndexInput freqIn;
+  IndexInput freqIn = null;
   IndexInput termsIn;
 
   private final StandardPositionsReader posReader;
@@ -49,8 +49,10 @@
   int maxSkipLevels;
 
   public StandardDocsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize) throws IOException {
-    freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, StandardCodec.FREQ_EXTENSION), readBufferSize);
-
+    String file = IndexFileNames.segmentFileName(segmentInfo.name, StandardCodec.FREQ_EXTENSION);
+    if(dir.fileExists(file)) {
+      freqIn = dir.openInput(file, readBufferSize);
+    }
     boolean success = false;
     try {
       if (segmentInfo.getHasProx()) {
@@ -64,7 +66,7 @@
       }
       success = true;
     } finally {
-      if (!success) {
+      if (!success & freqIn != null) {
         freqIn.close();
       }
     }
@@ -104,7 +106,9 @@
   @Override
   public void close() throws IOException {
     try {
-      freqIn.close();
+      if(freqIn != null) {
+        freqIn.close();
+      }
     } finally {
       if (posReader != null) {
         posReader.close();

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPositionsReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPositionsReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPositionsReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardPositionsReader.java Mon Nov 16 21:24:41 2009
@@ -32,13 +32,17 @@
 // this class interacts w/ a docsreader
 public class StandardPositionsReader extends StandardPositionsProducer {
   
-  final IndexInput proxIn;
+  IndexInput proxIn;
   IndexInput termsIn;
 
   public StandardPositionsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize) throws IOException {
     assert segmentInfo.getHasProx();
-    proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, StandardCodec.PROX_EXTENSION), readBufferSize);
+    String file = IndexFileNames.segmentFileName(segmentInfo.name, StandardCodec.PROX_EXTENSION);
+    if(dir.fileExists(file)) {
+      proxIn = dir.openInput(file, readBufferSize);
+    }
   }
+    
 
   @Override
   public void start(IndexInput termsIn) throws IOException {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java Mon Nov 16 21:24:41 2009
@@ -60,9 +60,18 @@
 
   public StandardTermsDictReader(StandardTermsIndexReader indexReader, Directory dir, FieldInfos fieldInfos, String segment, StandardDocsProducer docs, int readBufferSize)
     throws IOException {
-
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, StandardCodec.TERMS_EXTENSION), readBufferSize);
+    
     this.segment = segment;
+    this.docs = docs;
+    
+    String file = IndexFileNames.segmentFileName(segment, StandardCodec.TERMS_EXTENSION);
+    //nocommit
+    if(!dir.fileExists(file)) {
+      in = null;
+      return;
+    }
+    in = dir.openInput(file, readBufferSize);
+
 
     boolean success = false;
     try {
@@ -70,7 +79,7 @@
 
       final long dirOffset = in.readLong();
 
-      this.docs = docs;
+
       // Have DocsProducer init itself
       docs.start(in);
 
@@ -122,13 +131,19 @@
   public void close() throws IOException {
     try {
       try {
-        indexReader.close();
+        if(indexReader != null) {
+          indexReader.close();
+        }
       } finally {
-        in.close();
+        if(in != null) {
+          in.close();
+        }
       }
     } finally {
       try {
-        docs.close();
+        if(docs != null) {
+          docs.close();
+        }
       } finally {
         for(FieldReader field : fields.values()) {
           field.close();

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/queryParser/MultiFieldQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/queryParser/MultiFieldQueryParser.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/queryParser/MultiFieldQueryParser.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/queryParser/MultiFieldQueryParser.java Mon Nov 16 21:24:41 2009
@@ -96,7 +96,6 @@
     this.fields = fields;
   }
   
-  @Override
   protected Query getFieldQuery(String field, String queryText, int slop) throws ParseException {
     if (field == null) {
       List<BooleanClause> clauses = new ArrayList<BooleanClause>();
@@ -133,13 +132,11 @@
   }
   
 
-  @Override
   protected Query getFieldQuery(String field, String queryText) throws ParseException {
     return getFieldQuery(field, queryText, 0);
   }
 
 
-  @Override
   protected Query getFuzzyQuery(String field, String termStr, float minSimilarity) throws ParseException
   {
     if (field == null) {
@@ -153,7 +150,6 @@
     return super.getFuzzyQuery(field, termStr, minSimilarity);
   }
 
-  @Override
   protected Query getPrefixQuery(String field, String termStr) throws ParseException
   {
     if (field == null) {
@@ -167,7 +163,6 @@
     return super.getPrefixQuery(field, termStr);
   }
 
-  @Override
   protected Query getWildcardQuery(String field, String termStr) throws ParseException {
     if (field == null) {
       List<BooleanClause> clauses = new ArrayList<BooleanClause>();
@@ -181,7 +176,6 @@
   }
 
  
-  @Override
   protected Query getRangeQuery(String field, String part1, String part2, boolean inclusive) throws ParseException {
     if (field == null) {
       List<BooleanClause> clauses = new ArrayList<BooleanClause>();

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanQuery.java Mon Nov 16 21:24:41 2009
@@ -118,13 +118,8 @@
    * any specific clauses are required (or prohibited).  This number will
    * only be compared against the number of matching optional clauses.
    * </p>
-   * <p>
-   * EXPERT NOTE: Using this method may force collecting docs in order,
-   * regardless of whether setAllowDocsOutOfOrder(true) has been called.
-   * </p>
    *
    * @param min the number of optional clauses that must match
-   * @see #setAllowDocsOutOfOrder
    */
   public void setMinimumNumberShouldMatch(int min) {
     this.minNrShouldMatch = min;

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/BooleanScorer2.java Mon Nov 16 21:24:41 2009
@@ -261,7 +261,7 @@
           ? requiredCountingSumScorer // no prohibited
           : new ReqExclScorer(requiredCountingSumScorer,
                               ((prohibitedScorers.size() == 1)
-                                ? (Scorer) prohibitedScorers.get(0)
+                                ? prohibitedScorers.get(0)
                                 : new DisjunctionSumScorer(prohibitedScorers)));
   }
 
@@ -301,7 +301,6 @@
   public float score() throws IOException {
     coordinator.nrMatchers = 0;
     float sum = countingSumScorer.score();
-    assert coordinator.nrMatchers >= 0;
     return sum * coordinator.coordFactors[coordinator.nrMatchers];
   }
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ConjunctionScorer.java Mon Nov 16 21:24:41 2009
@@ -33,7 +33,7 @@
     this(similarity, scorers.toArray(new Scorer[scorers.size()]));
   }
 
-  public ConjunctionScorer(Similarity similarity, Scorer[] scorers) throws IOException {
+  public ConjunctionScorer(Similarity similarity, Scorer... scorers) throws IOException {
     super(similarity);
     this.scorers = scorers;
     coord = similarity.coord(scorers.length, scorers.length);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Mon Nov 16 21:24:41 2009
@@ -18,7 +18,6 @@
  */
 
 import java.util.List;
-import java.util.Iterator;
 import java.io.IOException;
 
 import org.apache.lucene.util.ScorerDocQueue;
@@ -31,7 +30,7 @@
   private final int nrScorers;
   
   /** The subscorers. */
-  protected final List subScorers;
+  protected final List<Scorer> subScorers;
   
   /** The minimum number of scorers that should match. */
   private final int minimumNrMatchers;
@@ -68,7 +67,7 @@
    * <br>When minimumNrMatchers equals the number of subScorers,
    * it more efficient to use <code>ConjunctionScorer</code>.
    */
-  public DisjunctionSumScorer( List subScorers, int minimumNrMatchers) throws IOException {
+  public DisjunctionSumScorer( List<Scorer> subScorers, int minimumNrMatchers) throws IOException {
     super(null);
     
     nrScorers = subScorers.size();
@@ -89,7 +88,7 @@
   /** Construct a <code>DisjunctionScorer</code>, using one as the minimum number
    * of matching subscorers.
    */
-  public DisjunctionSumScorer(List subScorers) throws IOException {
+  public DisjunctionSumScorer(List<Scorer> subScorers) throws IOException {
     this(subScorers, 1);
   }
 
@@ -97,11 +96,9 @@
    * initialize <code>scorerDocQueue</code>.
    */
   private void initScorerDocQueue() throws IOException {
-    Iterator si = subScorers.iterator();
     scorerDocQueue = new ScorerDocQueue(nrScorers);
-    while (si.hasNext()) {
-      Scorer se = (Scorer) si.next();
-      if (se.nextDoc() != NO_MORE_DOCS) { // doc() method will be used in scorerDocQueue.
+    for (Scorer se : subScorers) {
+      if (se.nextDoc() != NO_MORE_DOCS) {
         scorerDocQueue.insert(se);
       }
     }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DocIdSetIterator.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DocIdSetIterator.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DocIdSetIterator.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/DocIdSetIterator.java Mon Nov 16 21:24:41 2009
@@ -32,7 +32,7 @@
   
   /**
    * When returned by {@link #nextDoc()}, {@link #advance(int)} and
-   * {@link #doc()} it means there are no more docs in the iterator.
+   * {@link #docID()} it means there are no more docs in the iterator.
    */
   public static final int NO_MORE_DOCS = Integer.MAX_VALUE;
 

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldCacheTermsFilter.java Mon Nov 16 21:24:41 2009
@@ -97,7 +97,7 @@
   private String field;
   private String[] terms;
 
-  public FieldCacheTermsFilter(String field, String[] terms) {
+  public FieldCacheTermsFilter(String field, String... terms) {
     this.field = field;
     this.terms = terms;
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldDocSortedHitQueue.java Mon Nov 16 21:24:41 2009
@@ -32,11 +32,11 @@
  */
 class FieldDocSortedHitQueue extends PriorityQueue<FieldDoc> {
 
-  volatile SortField[] fields;
+  volatile SortField[] fields = null;
 
   // used in the case where the fields are sorted by locale
   // based strings
-  volatile Collator[] collators;
+  volatile Collator[] collators = null;
 
 
   /**
@@ -44,9 +44,7 @@
    * @param fields Fieldable names, in priority order (highest priority first).
    * @param size  The number of hits to retain.  Must be greater than zero.
    */
-  FieldDocSortedHitQueue (SortField[] fields, int size) {
-    this.fields = fields;
-    this.collators = hasCollators (fields);
+  FieldDocSortedHitQueue (int size) {
     initialize (size);
   }
 
@@ -56,10 +54,10 @@
    * This is to handle the case using ParallelMultiSearcher where the
    * original list contains AUTO and we don't know the actual sort
    * type until the values come back.  The fields can only be set once.
-   * This method is thread safe.
+   * This method should be synchronized external like all other PQ methods.
    * @param fields
    */
-  synchronized void setFields (SortField[] fields) {
+  void setFields (SortField[] fields) {
     this.fields = fields;
     this.collators = hasCollators (fields);
   }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldValueHitQueue.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldValueHitQueue.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldValueHitQueue.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FieldValueHitQueue.java Mon Nov 16 21:24:41 2009
@@ -36,20 +36,17 @@
  */
 public abstract class FieldValueHitQueue extends PriorityQueue<FieldValueHitQueue.Entry> {
 
-  final static class Entry {
+  final static class Entry extends ScoreDoc {
     int slot;
-    int docID;
-    float score;
 
-    Entry(int slot, int docID, float score) {
+    Entry(int slot, int doc, float score) {
+      super(doc, score);
       this.slot = slot;
-      this.docID = docID;
-      this.score = score;
     }
     
     @Override
     public String toString() {
-      return "slot:" + slot + " docID:" + docID + " score=" + score;
+      return "slot:" + slot + " " + super.toString();
     }
   }
 
@@ -97,7 +94,7 @@
       }
 
       // avoid random sort order that could lead to duplicates (bug #31241):
-      return hitA.docID > hitB.docID;
+      return hitA.doc > hitB.doc;
     }
 
   }
@@ -139,7 +136,7 @@
       }
 
       // avoid random sort order that could lead to duplicates (bug #31241):
-      return hitA.docID > hitB.docID;
+      return hitA.doc > hitB.doc;
     }
     
   }
@@ -214,7 +211,7 @@
       fields[i] = comparators[i].value(entry.slot);
     }
     //if (maxscore > 1.0f) doc.score /= maxscore;   // normalize scores
-    return new FieldDoc(entry.docID, entry.score, fields);
+    return new FieldDoc(entry.doc, entry.score, fields);
   }
 
   /** Returns the SortFields being used by this hit queue. */

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilterManager.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilterManager.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilterManager.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilterManager.java Mon Nov 16 21:24:41 2009
@@ -24,6 +24,8 @@
 import java.util.Map;
 import java.util.TreeSet;
 
+import org.apache.lucene.util.ThreadInterruptedException;
+
 /**
  * Filter caching singleton.  It can be used 
  * to save filters locally for reuse.
@@ -193,8 +195,7 @@
         try {
           Thread.sleep(cleanSleepTime);
         } catch (InterruptedException ie) {
-          Thread.currentThread().interrupt();
-          throw new RuntimeException(ie);
+          throw new ThreadInterruptedException(ie);
         }
       }
     }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredDocIdSet.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredDocIdSet.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredDocIdSet.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/FilteredDocIdSet.java Mon Nov 16 21:24:41 2009
@@ -67,7 +67,6 @@
    * @see DocIdSetIterator
    * @see FilteredDocIdSetIterator
    */
-  // @Override
   @Override
   public DocIdSetIterator iterator() throws IOException {
     return new FilteredDocIdSetIterator(_innerSet.iterator()) {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/IndexSearcher.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/IndexSearcher.java Mon Nov 16 21:24:41 2009
@@ -54,6 +54,16 @@
   protected int[] docStarts;
 
   /** Creates a searcher searching the index in the named
+   *  directory, with readOnly=true
+   * @throws CorruptIndexException if the index is corrupt
+   * @throws IOException if there is a low-level IO error
+   * @param path directory where IndexReader will be opened
+   */
+  public IndexSearcher(Directory path) throws CorruptIndexException, IOException {
+    this(IndexReader.open(path, true), true);
+  }
+
+  /** Creates a searcher searching the index in the named
    *  directory.  You should pass readOnly=true, since it
    *  gives much better concurrent performance, unless you
    *  intend to do write operations (delete documents or

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiPhraseQuery.java Mon Nov 16 21:24:41 2009
@@ -117,7 +117,7 @@
 
   // inherit javadoc
   @Override
-  public void extractTerms(Set terms) {
+  public void extractTerms(Set<Term> terms) {
     for (final Term[] arr : termArrays) {
       for (final Term term: arr) {
         terms.add(term);

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiSearcher.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiSearcher.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/MultiSearcher.java Mon Nov 16 21:24:41 2009
@@ -23,17 +23,20 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.DummyConcurrentLock;
 
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.locks.Lock;
 
 /** Implements search over a set of <code>Searchables</code>.
  *
- * <p>Applications usually need only call the inherited {@link #search(Query)}
- * or {@link #search(Query,Filter)} methods.
+ * <p>Applications usually need only call the inherited {@link #search(Query,int)}
+ * or {@link #search(Query,Filter,int)} methods.
  */
 public class MultiSearcher extends Searcher {
   
@@ -43,8 +46,8 @@
    * initialize Weights.
    */
   private static class CachedDfSource extends Searcher {
-    private Map<Term,Integer> dfMap; // Map from Terms to corresponding doc freqs
-    private int maxDoc; // document count
+    private final Map<Term,Integer> dfMap; // Map from Terms to corresponding doc freqs
+    private final int maxDoc; // document count
 
     public CachedDfSource(Map<Term,Integer> dfMap, int maxDoc, Similarity similarity) {
       this.dfMap = dfMap;
@@ -66,7 +69,7 @@
 
     @Override
     public int[] docFreqs(Term[] terms) {
-      int[] result = new int[terms.length];
+      final int[] result = new int[terms.length];
       for (int i = 0; i < terms.length; i++) {
         result[i] = docFreq(terms[i]);
       }
@@ -97,8 +100,9 @@
       throw new UnsupportedOperationException();
     }
     
+    @Override
     public Document doc(int i, FieldSelector fieldSelector) {
-        throw new UnsupportedOperationException();
+      throw new UnsupportedOperationException();
     }
 
     @Override
@@ -170,6 +174,7 @@
   }
 
   // inherit javadoc
+  @Override
   public Document doc(int n, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
     int i = subSearcher(n);			  // find searcher index
     return searchables[i].doc(n - starts[i], fieldSelector);	  // dispatch to searcher
@@ -196,22 +201,16 @@
   public TopDocs search(Weight weight, Filter filter, int nDocs)
       throws IOException {
 
-    HitQueue hq = new HitQueue(nDocs, false);
+    final HitQueue hq = new HitQueue(nDocs, false);
     int totalHits = 0;
 
     for (int i = 0; i < searchables.length; i++) { // search each searcher
-      TopDocs docs = searchables[i].search(weight, filter, nDocs);
-      totalHits += docs.totalHits;		  // update totalHits
-      ScoreDoc[] scoreDocs = docs.scoreDocs;
-      for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
-        ScoreDoc scoreDoc = scoreDocs[j];
-        scoreDoc.doc += starts[i];                // convert doc
-        if(scoreDoc == hq.insertWithOverflow(scoreDoc))
-          break;                                // no more scores > minScore
-      }
+      final TopDocs docs = new MultiSearcherCallableNoSort(DummyConcurrentLock.INSTANCE,
+        searchables[i], weight, filter, nDocs, hq, i, starts).call();
+      totalHits += docs.totalHits; // update totalHits
     }
 
-    ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
+    final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
     for (int i = hq.size()-1; i >= 0; i--)	  // put docs in array
       scoreDocs[i] = hq.pop();
     
@@ -221,41 +220,20 @@
   }
 
   @Override
-  public TopFieldDocs search (Weight weight, Filter filter, int n, Sort sort)
-  throws IOException {
-    FieldDocSortedHitQueue hq = null;
+  public TopFieldDocs search (Weight weight, Filter filter, int n, Sort sort) throws IOException {
+    FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue(n);
     int totalHits = 0;
 
     float maxScore=Float.NEGATIVE_INFINITY;
     
     for (int i = 0; i < searchables.length; i++) { // search each searcher
-      TopFieldDocs docs = searchables[i].search (weight, filter, n, sort);
-      // If one of the Sort fields is FIELD_DOC, need to fix its values, so that
-      // it will break ties by doc Id properly. Otherwise, it will compare to
-      // 'relative' doc Ids, that belong to two different searchers.
-      for (int j = 0; j < docs.fields.length; j++) {
-        if (docs.fields[j].getType() == SortField.DOC) {
-          // iterate over the score docs and change their fields value
-          for (int j2 = 0; j2 < docs.scoreDocs.length; j2++) {
-            FieldDoc fd = (FieldDoc) docs.scoreDocs[j2];
-            fd.fields[j] = Integer.valueOf(((Integer) fd.fields[j]).intValue() + starts[i]);
-          }
-          break;
-        }
-      }
-      if (hq == null) hq = new FieldDocSortedHitQueue (docs.fields, n);
-      totalHits += docs.totalHits;		  // update totalHits
+      final TopFieldDocs docs = new MultiSearcherCallableWithSort(DummyConcurrentLock.INSTANCE,
+        searchables[i], weight, filter, n, hq, sort, i, starts).call();
+      totalHits += docs.totalHits; // update totalHits
       maxScore = Math.max(maxScore, docs.getMaxScore());
-      ScoreDoc[] scoreDocs = docs.scoreDocs;
-      for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
-        ScoreDoc scoreDoc = scoreDocs[j];
-        scoreDoc.doc += starts[i];                // convert doc
-        if (scoreDoc == hq.insertWithOverflow((FieldDoc) scoreDoc))
-          break;                                  // no more scores > minScore
-      }
     }
 
-    ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
+    final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
     for (int i = hq.size() - 1; i >= 0; i--)	  // put docs in array
       scoreDocs[i] =  hq.pop();
 
@@ -295,7 +273,7 @@
 
   @Override
   public Query rewrite(Query original) throws IOException {
-    Query[] queries = new Query[searchables.length];
+    final Query[] queries = new Query[searchables.length];
     for (int i = 0; i < searchables.length; i++) {
       queries[i] = searchables[i].rewrite(original);
     }
@@ -304,7 +282,7 @@
 
   @Override
   public Explanation explain(Weight weight, int doc) throws IOException {
-    int i = subSearcher(doc);			  // find searcher index
+    final int i = subSearcher(doc);			  // find searcher index
     return searchables[i].explain(weight, doc - starts[i]); // dispatch to searcher
   }
 
@@ -326,14 +304,14 @@
   @Override
   protected Weight createWeight(Query original) throws IOException {
     // step 1
-    Query rewrittenQuery = rewrite(original);
+    final Query rewrittenQuery = rewrite(original);
 
     // step 2
-    Set<Term> terms = new HashSet<Term>();
+    final Set<Term> terms = new HashSet<Term>();
     rewrittenQuery.extractTerms(terms);
 
     // step3
-    Term[] allTermsArray = new Term[terms.size()];
+    final Term[] allTermsArray = new Term[terms.size()];
     terms.toArray(allTermsArray);
     int[] aggregatedDfs = new int[terms.size()];
     for (int i = 0; i < searchables.length; i++) {
@@ -343,16 +321,129 @@
       }
     }
 
-    HashMap<Term,Integer> dfMap = new HashMap<Term,Integer>();
+    final HashMap<Term,Integer> dfMap = new HashMap<Term,Integer>();
     for(int i=0; i<allTermsArray.length; i++) {
       dfMap.put(allTermsArray[i], Integer.valueOf(aggregatedDfs[i]));
     }
 
     // step4
-    int numDocs = maxDoc();
-    CachedDfSource cacheSim = new CachedDfSource(dfMap, numDocs, getSimilarity());
+    final int numDocs = maxDoc();
+    final CachedDfSource cacheSim = new CachedDfSource(dfMap, numDocs, getSimilarity());
 
     return rewrittenQuery.weight(cacheSim);
   }
 
+  /**
+   * A thread subclass for searching a single searchable 
+   */
+  static class MultiSearcherCallableNoSort implements Callable<TopDocs> {
+
+    private final Lock lock;
+    private final Searchable searchable;
+    private final Weight weight;
+    private final Filter filter;
+    private final int nDocs;
+    private final int i;
+    private final HitQueue hq;
+    private final int[] starts;
+
+    public MultiSearcherCallableNoSort(Lock lock, Searchable searchable, Weight weight,
+        Filter filter, int nDocs, HitQueue hq, int i, int[] starts) {
+      this.lock = lock;
+      this.searchable = searchable;
+      this.weight = weight;
+      this.filter = filter;
+      this.nDocs = nDocs;
+      this.hq = hq;
+      this.i = i;
+      this.starts = starts;
+    }
+
+    public TopDocs call() throws IOException {
+      final TopDocs docs = searchable.search (weight, filter, nDocs);
+      final ScoreDoc[] scoreDocs = docs.scoreDocs;
+      for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
+        final ScoreDoc scoreDoc = scoreDocs[j];
+        scoreDoc.doc += starts[i]; // convert doc 
+        //it would be so nice if we had a thread-safe insert 
+        lock.lock();
+        try {
+          if (scoreDoc == hq.insertWithOverflow(scoreDoc))
+            break;
+        } finally {
+          lock.unlock();
+        }
+      }
+      return docs;
+    }
+  }
+
+  /**
+   * A thread subclass for searching a single searchable 
+   */
+  static class MultiSearcherCallableWithSort implements Callable<TopFieldDocs> {
+
+    private final Lock lock;
+    private final Searchable searchable;
+    private final Weight weight;
+    private final Filter filter;
+    private final int nDocs;
+    private final int i;
+    private final FieldDocSortedHitQueue hq;
+    private final int[] starts;
+    private final Sort sort;
+
+    public MultiSearcherCallableWithSort(Lock lock, Searchable searchable, Weight weight,
+        Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort, int i, int[] starts) {
+      this.lock = lock;
+      this.searchable = searchable;
+      this.weight = weight;
+      this.filter = filter;
+      this.nDocs = nDocs;
+      this.hq = hq;
+      this.i = i;
+      this.starts = starts;
+      this.sort = sort;
+    }
+
+    public TopFieldDocs call() throws IOException {
+      final TopFieldDocs docs = searchable.search (weight, filter, nDocs, sort);
+      // If one of the Sort fields is FIELD_DOC, need to fix its values, so that
+      // it will break ties by doc Id properly. Otherwise, it will compare to
+      // 'relative' doc Ids, that belong to two different searchables.
+      for (int j = 0; j < docs.fields.length; j++) {
+        if (docs.fields[j].getType() == SortField.DOC) {
+          // iterate over the score docs and change their fields value
+          for (int j2 = 0; j2 < docs.scoreDocs.length; j2++) {
+            FieldDoc fd = (FieldDoc) docs.scoreDocs[j2];
+            fd.fields[j] = Integer.valueOf(((Integer) fd.fields[j]).intValue() + starts[i]);
+          }
+          break;
+        }
+      }
+
+      lock.lock();
+      try {
+        hq.setFields(docs.fields);
+      } finally {
+        lock.unlock();
+      }
+
+      final ScoreDoc[] scoreDocs = docs.scoreDocs;
+      for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
+        final FieldDoc fieldDoc = (FieldDoc) scoreDocs[j];
+        fieldDoc.doc += starts[i]; // convert doc 
+        //it would be so nice if we had a thread-safe insert 
+        lock.lock();
+        try {
+          if (fieldDoc == hq.insertWithOverflow(fieldDoc))
+            break;
+        } finally {
+          lock.unlock();
+        }
+      }
+      return docs;
+    }
+  }
+
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ParallelMultiSearcher.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ParallelMultiSearcher.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ParallelMultiSearcher.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/ParallelMultiSearcher.java Mon Nov 16 21:24:41 2009
@@ -18,80 +18,85 @@
  */
 
 import java.io.IOException;
+import java.util.List;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.Callable;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.NamedThreadFactory;
 import org.apache.lucene.util.PriorityQueue;
+import org.apache.lucene.util.ThreadInterruptedException;
 
 /** Implements parallel search over a set of <code>Searchables</code>.
  *
- * <p>Applications usually need only call the inherited {@link #search(Query)}
- * or {@link #search(Query,Filter)} methods.
+ * <p>Applications usually need only call the inherited {@link #search(Query,int)}
+ * or {@link #search(Query,Filter,int)} methods.
  */
 public class ParallelMultiSearcher extends MultiSearcher {
+  
+  private final ExecutorService executor;
+  private final Searchable[] searchables;
+  private final int[] starts;
 
-  private Searchable[] searchables;
-  private int[] starts;
-
-  /** Creates a searchable which searches <i>searchables</i>. */
+  /** Creates a {@link Searchable} which searches <i>searchables</i>. */
   public ParallelMultiSearcher(Searchable... searchables) throws IOException {
     super(searchables);
     this.searchables = searchables;
     this.starts = getStarts();
+    executor = Executors.newCachedThreadPool(new NamedThreadFactory(this.getClass().getSimpleName())); 
   }
 
   /**
-   * TODO: parallelize this one too
+   * Executes each {@link Searchable}'s docFreq() in its own thread and waits for each search to complete and merge
+   * the results back together.
    */
   @Override
-  public int docFreq(Term term) throws IOException {
-    return super.docFreq(term);
+  public int docFreq(final Term term) throws IOException {
+    @SuppressWarnings("unchecked") final Future<Integer>[] searchThreads = new Future[searchables.length];
+    for (int i = 0; i < searchables.length; i++) { // search each searchable
+      final Searchable searchable = searchables[i];
+      searchThreads[i] = executor.submit(new Callable<Integer>() {
+        public Integer call() throws IOException {
+          return Integer.valueOf(searchable.docFreq(term));
+        }
+      });
+    }
+    final CountDocFreq func = new CountDocFreq();
+    foreach(func, Arrays.asList(searchThreads));
+    return func.docFreq;
   }
 
   /**
-   * A search implementation which spans a new thread for each
-   * Searchable, waits for each search to complete and merge
+   * A search implementation which executes each 
+   * {@link Searchable} in its own thread and waits for each search to complete and merge
    * the results back together.
    */
   @Override
-  public TopDocs search(Weight weight, Filter filter, int nDocs)
-    throws IOException {
-    HitQueue hq = new HitQueue(nDocs, false);
-    int totalHits = 0;
-    MultiSearcherThread[] msta =
-      new MultiSearcherThread[searchables.length];
+  public TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
+    final HitQueue hq = new HitQueue(nDocs, false);
+    final Lock lock = new ReentrantLock();
+    @SuppressWarnings("unchecked") final Future<TopDocs>[] searchThreads = new Future[searchables.length];
     for (int i = 0; i < searchables.length; i++) { // search each searchable
-      // Assume not too many searchables and cost of creating a thread is by far inferior to a search
-      msta[i] = new MultiSearcherThread(searchables[i], weight, filter, nDocs,
-          hq, i, starts, "MultiSearcher thread #" + (i + 1));
-      msta[i].start();
+      searchThreads[i] = executor.submit(
+          new MultiSearcherCallableNoSort(lock, searchables[i], weight, filter, nDocs, hq, i, starts));
     }
 
-    for (int i = 0; i < searchables.length; i++) {
-      try {
-        msta[i].join();
-      } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
-      }
-      IOException ioe = msta[i].getIOException();
-      if (ioe == null) {
-        totalHits += msta[i].hits();
-      } else {
-        // if one search produced an IOException, rethrow it
-        throw ioe;
-      }
-    }
+    final CountTotalHits<TopDocs> func = new CountTotalHits<TopDocs>();
+    foreach(func, Arrays.asList(searchThreads));
 
-    ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
+    final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
     for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
       scoreDocs[i] = hq.pop();
 
-    float maxScore = (totalHits==0) ? Float.NEGATIVE_INFINITY : scoreDocs[0].score;
-    
-    return new TopDocs(totalHits, scoreDocs, maxScore);
+    return new TopDocs(func.totalHits, scoreDocs, func.maxScore);
   }
 
   /**
@@ -100,45 +105,25 @@
    * the results back together.
    */
   @Override
-  public TopFieldDocs search(Weight weight, Filter filter, int nDocs, Sort sort)
-    throws IOException {
-    // don't specify the fields - we'll wait to do this until we get results
-    FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue (null, nDocs);
-    int totalHits = 0;
-    MultiSearcherThread[] msta = new MultiSearcherThread[searchables.length];
+  public TopFieldDocs search(Weight weight, Filter filter, int nDocs, Sort sort) throws IOException {
+    if (sort == null) throw new NullPointerException();
+
+    final FieldDocSortedHitQueue hq = new FieldDocSortedHitQueue(nDocs);
+    final Lock lock = new ReentrantLock();
+    @SuppressWarnings("unchecked") final Future<TopFieldDocs>[] searchThreads = new Future[searchables.length];
     for (int i = 0; i < searchables.length; i++) { // search each searchable
-      // Assume not too many searchables and cost of creating a thread is by far inferior to a search
-      msta[i] = new MultiSearcherThread(searchables[i], weight, filter, nDocs,
-          hq, sort, i, starts, "MultiSearcher thread #" + (i + 1));
-      msta[i].start();
+      searchThreads[i] = executor.submit(
+          new MultiSearcherCallableWithSort(lock, searchables[i], weight, filter, nDocs, hq, sort, i, starts));
     }
 
-    float maxScore=Float.NEGATIVE_INFINITY;
-    
-    for (int i = 0; i < searchables.length; i++) {
-      try {
-        msta[i].join();
-      } catch (InterruptedException ie) {
-        // In 3.0 we will change this to throw
-        // InterruptedException instead
-        Thread.currentThread().interrupt();
-        throw new RuntimeException(ie);
-      }
-      IOException ioe = msta[i].getIOException();
-      if (ioe == null) {
-        totalHits += msta[i].hits();
-        maxScore=Math.max(maxScore, msta[i].getMaxScore());
-      } else {
-        // if one search produced an IOException, rethrow it
-        throw ioe;
-      }
-    }
+    final CountTotalHits<TopFieldDocs> func = new CountTotalHits<TopFieldDocs>();
+    foreach(func, Arrays.asList(searchThreads));
 
-    ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
+    final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
     for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
       scoreDocs[i] = hq.pop();
 
-    return new TopFieldDocs(totalHits, scoreDocs, hq.getFields(), maxScore);
+    return new TopFieldDocs(func.totalHits, scoreDocs, hq.getFields(), func.maxScore);
   }
 
   /** Lower-level search API.
@@ -147,17 +132,18 @@
   *
   * <p>Applications should only use this if they need <i>all</i> of the
   * matching documents.  The high-level search API ({@link
-  * Searcher#search(Query)}) is usually more efficient, as it skips
+  * Searcher#search(Query,int)}) is usually more efficient, as it skips
   * non-high-scoring hits.
+  * 
+  * <p>This method cannot be parallelized, because {@link Collector}
+  * supports no concurrent access.
   *
   * @param weight to match documents
   * @param filter if non-null, a bitset used to eliminate some documents
   * @param collector to receive hits
-  * 
-  * TODO: parallelize this one too
   */
   @Override
-  public void search(Weight weight, Filter filter, final Collector collector)
+  public void search(final Weight weight, final Filter filter, final Collector collector)
    throws IOException {
    for (int i = 0; i < searchables.length; i++) {
 
@@ -165,17 +151,17 @@
 
      final Collector hc = new Collector() {
        @Override
-       public void setScorer(Scorer scorer) throws IOException {
+       public void setScorer(final Scorer scorer) throws IOException {
          collector.setScorer(scorer);
        }
        
        @Override
-       public void collect(int doc) throws IOException {
+       public void collect(final int doc) throws IOException {
          collector.collect(doc);
        }
        
        @Override
-       public void setNextReader(IndexReader reader, int docBase) throws IOException {
+       public void setNextReader(final IndexReader reader, final int docBase) throws IOException {
          collector.setNextReader(reader, start + docBase);
        }
        
@@ -188,117 +174,57 @@
      searchables[i].search(weight, filter, hc);
    }
   }
-
+  
   /*
-   * TODO: this one could be parallelized too
-   * @see org.apache.lucene.search.Searchable#rewrite(org.apache.lucene.search.Query)
+   * apply the function to each element of the list. This method encapsulates the logic how 
+   * to wait for concurrently executed searchables.  
    */
-  @Override
-  public Query rewrite(Query original) throws IOException {
-    return super.rewrite(original);
-  }
-
-}
-
-/**
- * A thread subclass for searching a single searchable 
- */
-class MultiSearcherThread extends Thread {
-
-  private Searchable searchable;
-  private Weight weight;
-  private Filter filter;
-  private int nDocs;
-  private TopDocs docs;
-  private int i;
-  private PriorityQueue<? extends ScoreDoc> hq;
-  private int[] starts;
-  private IOException ioe;
-  private Sort sort;
-
-  public MultiSearcherThread(Searchable searchable, Weight weight, Filter filter,
-      int nDocs, HitQueue hq, int i, int[] starts, String name) {
-    super(name);
-    this.searchable = searchable;
-    this.weight = weight;
-    this.filter = filter;
-    this.nDocs = nDocs;
-    this.hq = hq;
-    this.i = i;
-    this.starts = starts;
-  }
-
-  public MultiSearcherThread(Searchable searchable, Weight weight,
-      Filter filter, int nDocs, FieldDocSortedHitQueue hq, Sort sort, int i,
-      int[] starts, String name) {
-    super(name);
-    this.searchable = searchable;
-    this.weight = weight;
-    this.filter = filter;
-    this.nDocs = nDocs;
-    this.hq = hq;
-    this.i = i;
-    this.starts = starts;
-    this.sort = sort;
-  }
-
-  @Override
-  @SuppressWarnings ("unchecked")
-  public void run() {
-    try {
-      docs = (sort == null) ? searchable.search (weight, filter, nDocs)
-        : searchable.search (weight, filter, nDocs, sort);
-    }
-    // Store the IOException for later use by the caller of this thread
-    catch (IOException ioe) {
-      this.ioe = ioe;
-    }
-    if (ioe == null) {
-      if (sort != null) {
-        TopFieldDocs docsFields = (TopFieldDocs) docs;
-        // If one of the Sort fields is FIELD_DOC, need to fix its values, so that
-        // it will break ties by doc Id properly. Otherwise, it will compare to
-        // 'relative' doc Ids, that belong to two different searchables.
-        for (int j = 0; j < docsFields.fields.length; j++) {
-          if (docsFields.fields[j].getType() == SortField.DOC) {
-            // iterate over the score docs and change their fields value
-            for (int j2 = 0; j2 < docs.scoreDocs.length; j2++) {
-              FieldDoc fd = (FieldDoc) docs.scoreDocs[j2];
-              fd.fields[j] = Integer.valueOf(((Integer) fd.fields[j]).intValue() + starts[i]);
-            }
-            break;
-          }
-        }
-
-        ((FieldDocSortedHitQueue) hq).setFields(docsFields.fields);
-      }
-      ScoreDoc[] scoreDocs = docs.scoreDocs;
-      for (int j = 0;
-           j < scoreDocs.length;
-           j++) { // merge scoreDocs into hq
-        ScoreDoc scoreDoc = scoreDocs[j];
-        scoreDoc.doc += starts[i]; // convert doc 
-        //it would be so nice if we had a thread-safe insert 
-        synchronized (hq) {
-          // this cast is bad, because we assume that the list has correct type.
-          // Because of that we have the @SuppressWarnings :-(
-          if (scoreDoc == ((PriorityQueue<ScoreDoc>) hq).insertWithOverflow(scoreDoc))
-            break;
-        } // no more scores > minScore
+  private <T> void foreach(Function<T> func, List<Future<T>> list) throws IOException{
+    for (Future<T> future : list) {
+      try{
+        func.apply(future.get());
+      } catch (ExecutionException e) {
+        if (e.getCause() instanceof IOException)
+          throw (IOException) e.getCause();
+        throw new RuntimeException(e.getCause());
+      } catch (InterruptedException ie) {
+        throw new ThreadInterruptedException(ie);
       }
     }
   }
 
-  public int hits() {
-    return docs.totalHits;
+  // Both functions could be reduced to Int as other values of TopDocs
+  // are not needed. Using sep. functions is more self documenting.
+  /**
+   * A function with one argument
+   * @param <T> the argument type
+   */
+  private static interface Function<T> {
+    abstract void apply(T t);
   }
 
-  public float getMaxScore() {
-      return docs.getMaxScore();
+  /**
+   * Counts the total number of hits for all {@link TopDocs} instances
+   * provided. 
+   */
+  private static final class CountTotalHits<T extends TopDocs> implements Function<T> {
+    int totalHits = 0;
+    float maxScore = Float.NEGATIVE_INFINITY;
+    
+    public void apply(T t) {
+      totalHits += t.totalHits;
+      maxScore = Math.max(maxScore, t.getMaxScore());
+    }
   }
-  
-  public IOException getIOException() {
-    return ioe;
+  /**
+   * Accumulates the document frequency for a term.
+   */
+  private static final class CountDocFreq implements Function<Integer>{
+    int docFreq = 0;
+    
+    public void apply(Integer t) {
+      docFreq += t.intValue();
+    }
   }
 
 }

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Query.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Query.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Query.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Query.java Mon Nov 16 21:24:41 2009
@@ -178,7 +178,7 @@
    *
    *<p>A utility for use by {@link #combine(Query[])} implementations.
    */
-  public static Query mergeBooleanQueries(BooleanQuery[] queries) {
+  public static Query mergeBooleanQueries(BooleanQuery... queries) {
     HashSet<BooleanClause> allClauses = new HashSet<BooleanClause>();
     for (BooleanQuery booleanQuery : queries) {
       for (BooleanClause clause : booleanQuery) {

Modified: lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Searchable.java
URL: http://svn.apache.org/viewvc/lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Searchable.java?rev=880963&r1=880962&r2=880963&view=diff
==============================================================================
--- lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Searchable.java (original)
+++ lucene/java/branches/flex_1458/src/java/org/apache/lucene/search/Searchable.java Mon Nov 16 21:24:41 2009
@@ -53,7 +53,7 @@
    * 
    * <p>
    * Applications should only use this if they need <i>all</i> of the matching
-   * documents. The high-level search API ({@link Searcher#search(Query)}) is
+   * documents. The high-level search API ({@link Searcher#search(Query,int)}) is
    * usually more efficient, as it skips non-high-scoring hits.
    * 
    * @param weight
@@ -93,8 +93,8 @@
   /** Expert: Low-level search implementation.  Finds the top <code>n</code>
    * hits for <code>query</code>, applying <code>filter</code> if non-null.
    *
-   * <p>Applications should usually call {@link Searcher#search(Query)} or
-   * {@link Searcher#search(Query,Filter)} instead.
+   * <p>Applications should usually call {@link Searcher#search(Query,int)} or
+   * {@link Searcher#search(Query,Filter,int)} instead.
    * @throws BooleanQuery.TooManyClauses
    */
   TopDocs search(Weight weight, Filter filter, int n) throws IOException;



Mime
View raw message