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