pylucene 3.5.0-3
[pylucene.git] / lucene-java-3.5.0 / lucene / src / java / org / apache / lucene / search / IndexSearcher.java
1 package org.apache.lucene.search;
2
3 /**
4  * Licensed to the Apache Software Foundation (ASF) under one or more
5  * contributor license agreements.  See the NOTICE file distributed with
6  * this work for additional information regarding copyright ownership.
7  * The ASF licenses this file to You under the Apache License, Version 2.0
8  * (the "License"); you may not use this file except in compliance with
9  * the License.  You may obtain a copy of the License at
10  *
11  *     http://www.apache.org/licenses/LICENSE-2.0
12  *
13  * Unless required by applicable law or agreed to in writing, software
14  * distributed under the License is distributed on an "AS IS" BASIS,
15  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16  * See the License for the specific language governing permissions and
17  * limitations under the License.
18  */
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.Iterator;
23 import java.util.List;
24 import java.util.NoSuchElementException;
25 import java.util.concurrent.Callable;
26 import java.util.concurrent.CompletionService;
27 import java.util.concurrent.ExecutionException;
28 import java.util.concurrent.Executor;
29 import java.util.concurrent.ExecutorCompletionService;
30 import java.util.concurrent.ExecutorService;
31 import java.util.concurrent.locks.Lock;
32 import java.util.concurrent.locks.ReentrantLock;
33
34 import org.apache.lucene.document.Document;
35 import org.apache.lucene.document.FieldSelector;
36 import org.apache.lucene.index.CorruptIndexException;
37 import org.apache.lucene.index.IndexReader;
38 import org.apache.lucene.index.Term;
39 import org.apache.lucene.store.Directory;
40 import org.apache.lucene.store.NIOFSDirectory; // javadocs
41 import org.apache.lucene.util.ReaderUtil;
42 import org.apache.lucene.util.ThreadInterruptedException;
43
44 /** Implements search over a single IndexReader.
45  *
46  * <p>Applications usually need only call the inherited
47  * {@link #search(Query,int)}
48  * or {@link #search(Query,Filter,int)} methods. For
49  * performance reasons, if your index is unchanging, you
50  * should share a single IndexSearcher instance across
51  * multiple searches instead of creating a new one
52  * per-search.  If your index has changed and you wish to
53  * see the changes reflected in searching, you should
54  * use {@link IndexReader#openIfChanged} to obtain a new reader and
55  * then create a new IndexSearcher from that.  Also, for
56  * low-latency turnaround it's best to use a near-real-time
57  * reader ({@link IndexReader#open(IndexWriter,boolean)}).
58  * Once you have a new {@link IndexReader}, it's relatively
59  * cheap to create a new IndexSearcher from it.
60  * 
61  * <a name="thread-safety"></a><p><b>NOTE</b>: <code>{@link
62  * IndexSearcher}</code> instances are completely
63  * thread safe, meaning multiple threads can call any of its
64  * methods, concurrently.  If your application requires
65  * external synchronization, you should <b>not</b>
66  * synchronize on the <code>IndexSearcher</code> instance;
67  * use your own (non-Lucene) objects instead.</p>
68  */
69 public class IndexSearcher extends Searcher {
70   IndexReader reader;
71   private boolean closeReader;
72   
73   // NOTE: these members might change in incompatible ways
74   // in the next release
75   protected final IndexReader[] subReaders;
76   protected final int[] docStarts;
77   
78   // These are only used for multi-threaded search
79   private final ExecutorService executor;
80   protected final IndexSearcher[] subSearchers;
81
82   private final int docBase;
83
84   /** Creates a searcher searching the index in the named
85    *  directory, with readOnly=true
86    * @param path directory where IndexReader will be opened
87    * @throws CorruptIndexException if the index is corrupt
88    * @throws IOException if there is a low-level IO error
89    * @deprecated use {@link IndexSearcher#IndexSearcher(IndexReader)} instead.
90    */
91   @Deprecated
92   public IndexSearcher(Directory path) throws CorruptIndexException, IOException {
93     this(IndexReader.open(path, true), true, null);
94   }
95
96   /** Creates a searcher searching the index in the named
97    *  directory.  You should pass readOnly=true, since it
98    *  gives much better concurrent performance, unless you
99    *  intend to do write operations (delete documents or
100    *  change norms) with the underlying IndexReader.
101    * @param path directory where IndexReader will be opened
102    * @param readOnly if true, the underlying IndexReader
103    * will be opened readOnly
104    * @throws CorruptIndexException if the index is corrupt
105    * @throws IOException if there is a low-level IO error
106    * @deprecated Use {@link IndexSearcher#IndexSearcher(IndexReader)} instead.
107    */
108   @Deprecated
109   public IndexSearcher(Directory path, boolean readOnly) throws CorruptIndexException, IOException {
110     this(IndexReader.open(path, readOnly), true, null);
111   }
112
113   /** Creates a searcher searching the provided index. */
114   public IndexSearcher(IndexReader r) {
115     this(r, false, null);
116   }
117
118   /** Runs searches for each segment separately, using the
119    *  provided ExecutorService.  IndexSearcher will not
120    *  shutdown/awaitTermination this ExecutorService on
121    *  close; you must do so, eventually, on your own.  NOTE:
122    *  if you are using {@link NIOFSDirectory}, do not use
123    *  the shutdownNow method of ExecutorService as this uses
124    *  Thread.interrupt under-the-hood which can silently
125    *  close file descriptors (see <a
126    *  href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
127    * 
128    * @lucene.experimental */
129   public IndexSearcher(IndexReader r, ExecutorService executor) {
130     this(r, false, executor);
131   }
132
133   /** Expert: directly specify the reader, subReaders and
134    *  their docID starts.
135    * 
136    * @lucene.experimental */
137   public IndexSearcher(IndexReader reader, IndexReader[] subReaders, int[] docStarts) {
138     this(reader, subReaders, docStarts, null);
139   }
140   
141   // Used only when we are an atomic sub-searcher in a parent
142   // IndexSearcher that has an ExecutorService, to record
143   // our docBase in the parent IndexSearcher:
144   private IndexSearcher(IndexReader r, int docBase) {
145     reader = r;
146     this.executor = null;
147     closeReader = false;
148     this.docBase = docBase;
149     subReaders = new IndexReader[] {r};
150     docStarts = new int[] {0};
151     subSearchers = null;
152   }
153
154   /** Expert: directly specify the reader, subReaders and
155    *  their docID starts, and an ExecutorService.  In this
156    *  case, each segment will be separately searched using the
157    *  ExecutorService.  IndexSearcher will not
158    *  shutdown/awaitTermination this ExecutorService on
159    *  close; you must do so, eventually, on your own.  NOTE:
160    *  if you are using {@link NIOFSDirectory}, do not use
161    *  the shutdownNow method of ExecutorService as this uses
162    *  Thread.interrupt under-the-hood which can silently
163    *  close file descriptors (see <a
164    *  href="https://issues.apache.org/jira/browse/LUCENE-2239">LUCENE-2239</a>).
165    * 
166    * @lucene.experimental */
167   public IndexSearcher(IndexReader reader, IndexReader[] subReaders, int[] docStarts, ExecutorService executor) {
168     this.reader = reader;
169     this.subReaders = subReaders;
170     this.docStarts = docStarts;
171     if (executor == null) {
172       subSearchers = null;
173     } else {
174       subSearchers = new IndexSearcher[subReaders.length];
175       for(int i=0;i<subReaders.length;i++) {
176         subSearchers[i] = new IndexSearcher(subReaders[i], docStarts[i]);
177       }
178     }
179     closeReader = false;
180     this.executor = executor;
181     docBase = 0;
182   }
183
184   private IndexSearcher(IndexReader r, boolean closeReader, ExecutorService executor) {
185     reader = r;
186     this.executor = executor;
187     this.closeReader = closeReader;
188
189     List<IndexReader> subReadersList = new ArrayList<IndexReader>();
190     gatherSubReaders(subReadersList, reader);
191     subReaders = subReadersList.toArray(new IndexReader[subReadersList.size()]);
192     docStarts = new int[subReaders.length];
193     int maxDoc = 0;
194     for (int i = 0; i < subReaders.length; i++) {
195       docStarts[i] = maxDoc;
196       maxDoc += subReaders[i].maxDoc();
197     }
198     if (executor == null) {
199       subSearchers = null;
200     } else {
201       subSearchers = new IndexSearcher[subReaders.length];
202       for (int i = 0; i < subReaders.length; i++) {
203         subSearchers[i] = new IndexSearcher(subReaders[i], docStarts[i]);
204       }
205     }
206     docBase = 0;
207   }
208
209   protected void gatherSubReaders(List<IndexReader> allSubReaders, IndexReader r) {
210     ReaderUtil.gatherSubReaders(allSubReaders, r);
211   }
212
213   /** Return the {@link IndexReader} this searches. */
214   public IndexReader getIndexReader() {
215     return reader;
216   }
217
218   /** Returns the atomic subReaders used by this searcher. */
219   public IndexReader[] getSubReaders() {
220     return subReaders;
221   }
222
223   /** Expert: Returns one greater than the largest possible document number.
224    * 
225    * @see org.apache.lucene.index.IndexReader#maxDoc()
226    */
227   @Override
228   public int maxDoc() {
229     return reader.maxDoc();
230   }
231
232   /** Returns total docFreq for this term. */
233   @Override
234   public int docFreq(final Term term) throws IOException {
235     if (executor == null) {
236       return reader.docFreq(term);
237     } else {
238       final ExecutionHelper<Integer> runner = new ExecutionHelper<Integer>(executor);
239       for(int i = 0; i < subReaders.length; i++) {
240         final IndexSearcher searchable = subSearchers[i];
241         runner.submit(new Callable<Integer>() {
242             public Integer call() throws IOException {
243               return Integer.valueOf(searchable.docFreq(term));
244             }
245           });
246       }
247       int docFreq = 0;
248       for (Integer num : runner) {
249         docFreq += num.intValue();
250       }
251       return docFreq;
252     }
253   }
254
255   /* Sugar for .getIndexReader().document(docID) */
256   @Override
257   public Document doc(int docID) throws CorruptIndexException, IOException {
258     return reader.document(docID);
259   }
260   
261   /* Sugar for .getIndexReader().document(docID, fieldSelector) */
262   @Override
263   public Document doc(int docID, FieldSelector fieldSelector) throws CorruptIndexException, IOException {
264     return reader.document(docID, fieldSelector);
265   }
266   
267   /** Expert: Set the Similarity implementation used by this Searcher.
268    *
269    * @see Similarity#setDefault(Similarity)
270    */
271   @Override
272   public void setSimilarity(Similarity similarity) {
273     super.setSimilarity(similarity);
274   }
275
276   @Override
277   public Similarity getSimilarity() {
278     return super.getSimilarity();
279   }
280
281   /**
282    * Note that the underlying IndexReader is not closed, if
283    * IndexSearcher was constructed with IndexSearcher(IndexReader r).
284    * If the IndexReader was supplied implicitly by specifying a directory, then
285    * the IndexReader is closed.
286    */
287   @Override
288   public void close() throws IOException {
289     if (closeReader) {
290       reader.close();
291     }
292   }
293
294   /** Finds the top <code>n</code>
295    * hits for <code>query</code> where all results are after a previous 
296    * result (<code>after</code>).
297    * <p>
298    * By passing the bottom result from a previous page as <code>after</code>,
299    * this method can be used for efficient 'deep-paging' across potentially
300    * large result sets.
301    *
302    * @throws BooleanQuery.TooManyClauses
303    */
304   public TopDocs searchAfter(ScoreDoc after, Query query, int n) throws IOException {
305     return searchAfter(after, query, null, n);
306   }
307   
308   /** Finds the top <code>n</code>
309    * hits for <code>query</code>, applying <code>filter</code> if non-null,
310    * where all results are after a previous result (<code>after</code>).
311    * <p>
312    * By passing the bottom result from a previous page as <code>after</code>,
313    * this method can be used for efficient 'deep-paging' across potentially
314    * large result sets.
315    *
316    * @throws BooleanQuery.TooManyClauses
317    */
318   public TopDocs searchAfter(ScoreDoc after, Query query, Filter filter, int n) throws IOException {
319     return search(createNormalizedWeight(query), filter, after, n);
320   }
321   
322   /** Finds the top <code>n</code>
323    * hits for <code>query</code>.
324    *
325    * @throws BooleanQuery.TooManyClauses
326    */
327   @Override
328   public TopDocs search(Query query, int n)
329     throws IOException {
330     return search(query, null, n);
331   }
332
333
334   /** Finds the top <code>n</code>
335    * hits for <code>query</code>, applying <code>filter</code> if non-null.
336    *
337    * @throws BooleanQuery.TooManyClauses
338    */
339   @Override
340   public TopDocs search(Query query, Filter filter, int n)
341     throws IOException {
342     return search(createNormalizedWeight(query), filter, n);
343   }
344
345   /** Lower-level search API.
346    *
347    * <p>{@link Collector#collect(int)} is called for every matching
348    * document.
349    * <br>Collector-based access to remote indexes is discouraged.
350    *
351    * <p>Applications should only use this if they need <i>all</i> of the
352    * matching documents.  The high-level search API ({@link
353    * Searcher#search(Query, Filter, int)}) is usually more efficient, as it skips
354    * non-high-scoring hits.
355    *
356    * @param query to match documents
357    * @param filter if non-null, used to permit documents to be collected.
358    * @param results to receive hits
359    * @throws BooleanQuery.TooManyClauses
360    */
361   @Override
362   public void search(Query query, Filter filter, Collector results)
363     throws IOException {
364     search(createNormalizedWeight(query), filter, results);
365   }
366
367   /** Lower-level search API.
368   *
369   * <p>{@link Collector#collect(int)} is called for every matching document.
370   *
371   * <p>Applications should only use this if they need <i>all</i> of the
372   * matching documents.  The high-level search API ({@link
373   * Searcher#search(Query, int)}) is usually more efficient, as it skips
374   * non-high-scoring hits.
375   * <p>Note: The <code>score</code> passed to this method is a raw score.
376   * In other words, the score will not necessarily be a float whose value is
377   * between 0 and 1.
378   * @throws BooleanQuery.TooManyClauses
379   */
380   @Override
381   public void search(Query query, Collector results)
382     throws IOException {
383     search(createNormalizedWeight(query), null, results);
384   }
385   
386   /** Search implementation with arbitrary sorting.  Finds
387    * the top <code>n</code> hits for <code>query</code>, applying
388    * <code>filter</code> if non-null, and sorting the hits by the criteria in
389    * <code>sort</code>.
390    * 
391    * <p>NOTE: this does not compute scores by default; use
392    * {@link IndexSearcher#setDefaultFieldSortScoring} to
393    * enable scoring.
394    *
395    * @throws BooleanQuery.TooManyClauses
396    */
397   @Override
398   public TopFieldDocs search(Query query, Filter filter, int n,
399                              Sort sort) throws IOException {
400     return search(createNormalizedWeight(query), filter, n, sort);
401   }
402
403   /**
404    * Search implementation with arbitrary sorting and no filter.
405    * @param query The query to search for
406    * @param n Return only the top n results
407    * @param sort The {@link org.apache.lucene.search.Sort} object
408    * @return The top docs, sorted according to the supplied {@link org.apache.lucene.search.Sort} instance
409    * @throws IOException
410    */
411   @Override
412   public TopFieldDocs search(Query query, int n,
413                              Sort sort) throws IOException {
414     return search(createNormalizedWeight(query), null, n, sort);
415   }
416
417   /** Expert: Low-level search implementation.  Finds the top <code>n</code>
418    * hits for <code>query</code>, applying <code>filter</code> if non-null.
419    *
420    * <p>Applications should usually call {@link Searcher#search(Query,int)} or
421    * {@link Searcher#search(Query,Filter,int)} instead.
422    * @throws BooleanQuery.TooManyClauses
423    */
424   @Override
425   public TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
426     return search(weight, filter, null, nDocs);
427   }
428   
429   /**
430    * Expert: Low-level search implementation.  Finds the top <code>n</code>
431    * hits for <code>query</code>, applying <code>filter</code> if non-null,
432    * returning results after <code>after</code>.
433    * 
434    * @throws BooleanQuery.TooManyClauses
435    */
436   protected TopDocs search(Weight weight, Filter filter, ScoreDoc after, int nDocs) throws IOException {
437     if (executor == null) {
438       // single thread
439       int limit = reader.maxDoc();
440       if (limit == 0) {
441         limit = 1;
442       }
443       nDocs = Math.min(nDocs, limit);
444       TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, after, !weight.scoresDocsOutOfOrder());
445       search(weight, filter, collector);
446       return collector.topDocs();
447     } else {
448       final HitQueue hq = new HitQueue(nDocs, false);
449       final Lock lock = new ReentrantLock();
450       final ExecutionHelper<TopDocs> runner = new ExecutionHelper<TopDocs>(executor);
451     
452       for (int i = 0; i < subReaders.length; i++) { // search each sub
453         runner.submit(
454                       new MultiSearcherCallableNoSort(lock, subSearchers[i], weight, filter, after, nDocs, hq));
455       }
456
457       int totalHits = 0;
458       float maxScore = Float.NEGATIVE_INFINITY;
459       for (final TopDocs topDocs : runner) {
460         if(topDocs.totalHits != 0) {
461           totalHits += topDocs.totalHits;
462           maxScore = Math.max(maxScore, topDocs.getMaxScore());
463         }
464       }
465
466       final ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
467       for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
468         scoreDocs[i] = hq.pop();
469
470       return new TopDocs(totalHits, scoreDocs, maxScore);
471     }
472   }
473
474   /** Expert: Low-level search implementation with arbitrary sorting.  Finds
475    * the top <code>n</code> hits for <code>query</code>, applying
476    * <code>filter</code> if non-null, and sorting the hits by the criteria in
477    * <code>sort</code>.
478    *
479    * <p>Applications should usually call {@link
480    * Searcher#search(Query,Filter,int,Sort)} instead.
481    * 
482    * @throws BooleanQuery.TooManyClauses
483    */
484   @Override
485   public TopFieldDocs search(Weight weight, Filter filter,
486       final int nDocs, Sort sort) throws IOException {
487     return search(weight, filter, nDocs, sort, true);
488   }
489
490   /**
491    * Just like {@link #search(Weight, Filter, int, Sort)}, but you choose
492    * whether or not the fields in the returned {@link FieldDoc} instances should
493    * be set by specifying fillFields.
494    *
495    * <p>NOTE: this does not compute scores by default.  If you
496    * need scores, create a {@link TopFieldCollector}
497    * instance by calling {@link TopFieldCollector#create} and
498    * then pass that to {@link #search(Weight, Filter,
499    * Collector)}.</p>
500    */
501   protected TopFieldDocs search(Weight weight, Filter filter, int nDocs,
502                                 Sort sort, boolean fillFields)
503       throws IOException {
504
505     if (sort == null) throw new NullPointerException();
506
507     if (executor == null) {
508       // single thread
509       int limit = reader.maxDoc();
510       if (limit == 0) {
511         limit = 1;
512       }
513       nDocs = Math.min(nDocs, limit);
514
515       TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
516                                                              fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());
517       search(weight, filter, collector);
518       return (TopFieldDocs) collector.topDocs();
519     } else {
520       final TopFieldCollector topCollector = TopFieldCollector.create(sort, nDocs,
521                                                                       fillFields,
522                                                                       fieldSortDoTrackScores,
523                                                                       fieldSortDoMaxScore,
524                                                                       false);
525
526       final Lock lock = new ReentrantLock();
527       final ExecutionHelper<TopFieldDocs> runner = new ExecutionHelper<TopFieldDocs>(executor);
528       for (int i = 0; i < subReaders.length; i++) { // search each sub
529         runner.submit(
530                       new MultiSearcherCallableWithSort(lock, subSearchers[i], weight, filter, nDocs, topCollector, sort));
531       }
532       int totalHits = 0;
533       float maxScore = Float.NEGATIVE_INFINITY;
534       for (final TopFieldDocs topFieldDocs : runner) {
535         if (topFieldDocs.totalHits != 0) {
536           totalHits += topFieldDocs.totalHits;
537           maxScore = Math.max(maxScore, topFieldDocs.getMaxScore());
538         }
539       }
540
541       final TopFieldDocs topDocs = (TopFieldDocs) topCollector.topDocs();
542
543       return new TopFieldDocs(totalHits, topDocs.scoreDocs, topDocs.fields, topDocs.getMaxScore());
544     }
545   }
546
547   /**
548    * Lower-level search API.
549    * 
550    * <p>
551    * {@link Collector#collect(int)} is called for every document. <br>
552    * Collector-based access to remote indexes is discouraged.
553    * 
554    * <p>
555    * Applications should only use this if they need <i>all</i> of the matching
556    * documents. The high-level search API ({@link Searcher#search(Query,int)}) is
557    * usually more efficient, as it skips non-high-scoring hits.
558    * 
559    * @param weight
560    *          to match documents
561    * @param filter
562    *          if non-null, used to permit documents to be collected.
563    * @param collector
564    *          to receive hits
565    * @throws BooleanQuery.TooManyClauses
566    */
567   @Override
568   public void search(Weight weight, Filter filter, Collector collector)
569       throws IOException {
570
571     // TODO: should we make this
572     // threaded...?  the Collector could be sync'd?
573
574     // always use single thread:
575     for (int i = 0; i < subReaders.length; i++) { // search each subreader
576       collector.setNextReader(subReaders[i], docBase + docStarts[i]);
577       final Scorer scorer = (filter == null) ?
578         weight.scorer(subReaders[i], !collector.acceptsDocsOutOfOrder(), true) :
579         FilteredQuery.getFilteredScorer(subReaders[i], getSimilarity(), weight, weight, filter);
580       if (scorer != null) {
581         scorer.score(collector);
582       }
583     }
584   }
585
586   /** Expert: called to re-write queries into primitive queries.
587    * @throws BooleanQuery.TooManyClauses
588    */
589   @Override
590   public Query rewrite(Query original) throws IOException {
591     Query query = original;
592     for (Query rewrittenQuery = query.rewrite(reader); rewrittenQuery != query;
593          rewrittenQuery = query.rewrite(reader)) {
594       query = rewrittenQuery;
595     }
596     return query;
597   }
598
599   /** Returns an Explanation that describes how <code>doc</code> scored against
600    * <code>query</code>.
601    *
602    * <p>This is intended to be used in developing Similarity implementations,
603    * and, for good performance, should not be displayed with every hit.
604    * Computing an explanation is as expensive as executing the query over the
605    * entire index.
606    */
607   @Override
608   public Explanation explain(Query query, int doc) throws IOException {
609     return explain(createNormalizedWeight(query), doc);
610   }
611
612   /** Expert: low-level implementation method
613    * Returns an Explanation that describes how <code>doc</code> scored against
614    * <code>weight</code>.
615    *
616    * <p>This is intended to be used in developing Similarity implementations,
617    * and, for good performance, should not be displayed with every hit.
618    * Computing an explanation is as expensive as executing the query over the
619    * entire index.
620    * <p>Applications should call {@link Searcher#explain(Query, int)}.
621    * @throws BooleanQuery.TooManyClauses
622    */
623   @Override
624   public Explanation explain(Weight weight, int doc) throws IOException {
625     int n = ReaderUtil.subIndex(doc, docStarts);
626     int deBasedDoc = doc - docStarts[n];
627     
628     return weight.explain(subReaders[n], deBasedDoc);
629   }
630
631   private boolean fieldSortDoTrackScores;
632   private boolean fieldSortDoMaxScore;
633
634   /** By default, no scores are computed when sorting by
635    *  field (using {@link #search(Query,Filter,int,Sort)}).
636    *  You can change that, per IndexSearcher instance, by
637    *  calling this method.  Note that this will incur a CPU
638    *  cost.
639    * 
640    *  @param doTrackScores If true, then scores are
641    *  returned for every matching document in {@link
642    *  TopFieldDocs}.
643    *
644    *  @param doMaxScore If true, then the max score for all
645    *  matching docs is computed. */
646   public void setDefaultFieldSortScoring(boolean doTrackScores, boolean doMaxScore) {
647     fieldSortDoTrackScores = doTrackScores;
648     fieldSortDoMaxScore = doMaxScore;
649     if (subSearchers != null) { // propagate settings to subs
650       for (IndexSearcher sub : subSearchers) {
651         sub.setDefaultFieldSortScoring(doTrackScores, doMaxScore);
652       }
653     }
654   }
655
656   /**
657    * Creates a normalized weight for a top-level {@link Query}.
658    * The query is rewritten by this method and {@link Query#createWeight} called,
659    * afterwards the {@link Weight} is normalized. The returned {@code Weight}
660    * can then directly be used to get a {@link Scorer}.
661    * @lucene.internal
662    */
663   public Weight createNormalizedWeight(Query query) throws IOException {
664     return super.createNormalizedWeight(query);
665   }
666
667   /**
668    * A thread subclass for searching a single searchable 
669    */
670   private static final class MultiSearcherCallableNoSort implements Callable<TopDocs> {
671
672     private final Lock lock;
673     private final IndexSearcher searchable;
674     private final Weight weight;
675     private final Filter filter;
676     private final ScoreDoc after;
677     private final int nDocs;
678     private final HitQueue hq;
679
680     public MultiSearcherCallableNoSort(Lock lock, IndexSearcher searchable, Weight weight,
681         Filter filter, ScoreDoc after, int nDocs, HitQueue hq) {
682       this.lock = lock;
683       this.searchable = searchable;
684       this.weight = weight;
685       this.filter = filter;
686       this.after = after;
687       this.nDocs = nDocs;
688       this.hq = hq;
689     }
690
691     public TopDocs call() throws IOException {
692       final TopDocs docs;
693       // we could call the 4-arg method, but we want to invoke the old method
694       // for backwards purposes unless someone is using the new searchAfter.
695       if (after == null) {
696         docs = searchable.search (weight, filter, nDocs);
697       } else {
698         docs = searchable.search (weight, filter, after, nDocs);
699       }
700       final ScoreDoc[] scoreDocs = docs.scoreDocs;
701       //it would be so nice if we had a thread-safe insert 
702       lock.lock();
703       try {
704         for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
705           final ScoreDoc scoreDoc = scoreDocs[j];
706           if (scoreDoc == hq.insertWithOverflow(scoreDoc)) {
707             break;
708           }
709         }
710       } finally {
711         lock.unlock();
712       }
713       return docs;
714     }
715   }
716
717
718   /**
719    * A thread subclass for searching a single searchable 
720    */
721   private static final class MultiSearcherCallableWithSort implements Callable<TopFieldDocs> {
722
723     private final Lock lock;
724     private final IndexSearcher searchable;
725     private final Weight weight;
726     private final Filter filter;
727     private final int nDocs;
728     private final TopFieldCollector hq;
729     private final Sort sort;
730
731     public MultiSearcherCallableWithSort(Lock lock, IndexSearcher searchable, Weight weight,
732                                          Filter filter, int nDocs, TopFieldCollector hq, Sort sort) {
733       this.lock = lock;
734       this.searchable = searchable;
735       this.weight = weight;
736       this.filter = filter;
737       this.nDocs = nDocs;
738       this.hq = hq;
739       this.sort = sort;
740     }
741
742     private final class FakeScorer extends Scorer {
743       float score;
744       int doc;
745
746       public FakeScorer() {
747         super(null, null);
748       }
749     
750       @Override
751       public int advance(int target) {
752         throw new UnsupportedOperationException();
753       }
754
755       @Override
756       public int docID() {
757         return doc;
758       }
759
760       @Override
761       public float freq() {
762         throw new UnsupportedOperationException();
763       }
764
765       @Override
766       public int nextDoc() {
767         throw new UnsupportedOperationException();
768       }
769     
770       @Override
771       public float score() {
772         return score;
773       }
774     }
775
776     private final FakeScorer fakeScorer = new FakeScorer();
777
778     public TopFieldDocs call() throws IOException {
779       final TopFieldDocs docs = searchable.search (weight, filter, nDocs, sort);
780       // If one of the Sort fields is FIELD_DOC, need to fix its values, so that
781       // it will break ties by doc Id properly. Otherwise, it will compare to
782       // 'relative' doc Ids, that belong to two different searchables.
783       for (int j = 0; j < docs.fields.length; j++) {
784         if (docs.fields[j].getType() == SortField.DOC) {
785           // iterate over the score docs and change their fields value
786           for (int j2 = 0; j2 < docs.scoreDocs.length; j2++) {
787             FieldDoc fd = (FieldDoc) docs.scoreDocs[j2];
788             fd.fields[j] = Integer.valueOf(((Integer) fd.fields[j]).intValue());
789           }
790           break;
791         }
792       }
793
794       lock.lock();
795       try {
796         hq.setNextReader(searchable.getIndexReader(), searchable.docBase);
797         hq.setScorer(fakeScorer);
798         for(ScoreDoc scoreDoc : docs.scoreDocs) {
799           final int docID = scoreDoc.doc - searchable.docBase;
800           fakeScorer.doc = docID;
801           fakeScorer.score = scoreDoc.score;
802           hq.collect(docID);
803         }
804       } finally {
805         lock.unlock();
806       }
807
808       return docs;
809     }
810   }
811
812   /**
813    * A helper class that wraps a {@link CompletionService} and provides an
814    * iterable interface to the completed {@link Callable} instances.
815    * 
816    * @param <T>
817    *          the type of the {@link Callable} return value
818    */
819   private static final class ExecutionHelper<T> implements Iterator<T>, Iterable<T> {
820     private final CompletionService<T> service;
821     private int numTasks;
822
823     ExecutionHelper(final Executor executor) {
824       this.service = new ExecutorCompletionService<T>(executor);
825     }
826
827     public boolean hasNext() {
828       return numTasks > 0;
829     }
830
831     public void submit(Callable<T> task) {
832       this.service.submit(task);
833       ++numTasks;
834     }
835
836     public T next() {
837       if(!this.hasNext())
838         throw new NoSuchElementException();
839       try {
840         return service.take().get();
841       } catch (InterruptedException e) {
842         throw new ThreadInterruptedException(e);
843       } catch (ExecutionException e) {
844         throw new RuntimeException(e);
845       } finally {
846         --numTasks;
847       }
848     }
849
850     public void remove() {
851       throw new UnsupportedOperationException();
852     }
853
854     public Iterator<T> iterator() {
855       // use the shortcut here - this is only used in a private context
856       return this;
857     }
858   }
859
860   @Override
861   public String toString() {
862     return "IndexSearcher(" + reader + ")";
863   }
864 }