Details
-
Improvement
-
Status: Closed
-
Minor
-
Resolution: Fixed
-
None
-
None
-
None
-
Operating System: other
Platform: Other
-
26120
Description
The following parallelizes searches among the various searchables in a
MultiSearcher.
The patch contains an inherited MultiSearcher, a unit test class and a light
modification to testMultiSearch to allow subclassing it.
I would be interested to know if somebody tried this on a multiple HDD
machine or in a distributed index architecture if performance improvement is
noticeable.
Jeff
Index: java/org/apache/lucene/search/ParallelMultiSearcher.java
===================================================================
RCS file: java/org/apache/lucene/search/ParallelMultiSearcher.java
diff -N java/org/apache/lucene/search/ParallelMultiSearcher.java
— /dev/null 1 Jan 1970 00:00:00 -0000
+++ java/org/apache/lucene/search/ParallelMultiSearcher.java 8 Jan 2004
18:00:00 -0000
@@ -0,0 +1,247 @@
+package org.apache.lucene.search;
+
+/* ====================================================================
+ * The Apache Software License, Version 1.1
+ *
+ * Copyright (c) 2001 The Apache Software Foundation. All rights
+ * reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *
+ * 2. Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in
+ * the documentation and/or other materials provided with the
+ * distribution.
+ *
+ * 3. The end-user documentation included with the redistribution,
+ * if any, must include the following acknowledgment:
+ * "This product includes software developed by the
+ * Apache Software Foundation (http://www.apache.org/)."
+ * Alternately, this acknowledgment may appear in the software itself,
+ * if and wherever such third-party acknowledgments normally appear.
+ *
+ * 4. The names "Apache" and "Apache Software Foundation" and
+ * "Apache Lucene" must not be used to endorse or promote products
+ * derived from this software without prior written permission. For
+ * written permission, please contact apache@apache.org.
+ *
+ * 5. Products derived from this software may not be called "Apache",
+ * "Apache Lucene", nor may "Apache" appear in their name, without
+ * prior written permission of the Apache Software Foundation.
+ *
+ * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
+ * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+ * DISCLAIMED. IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
+ * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
+ * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+ * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
+ * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+ * SUCH DAMAGE.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation. For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.Term;
+
+/** Implements parallel search over a set of <code>Searchables</code>.
+ *
+ * <p>Applications usually need only call the inherited
+ * or
{@link #search(Query,Filter)} methods.
+ */
+public class ParallelMultiSearcher extends MultiSearcher
+
+ /**
+ * TODO: parallelize this one too
+ */
+ public int docFreq(Term term) throws IOException
+
+ /**
+ * A search implementation which spans a new thread for each
+ * Searchable, waits for each search to complete and merge
+ * the results back together.
+ */
+ public TopDocs search(Query query, Filter filter, int nDocs)
+ throws IOException {
+ HitQueue hq = new HitQueue(nDocs);
+ int totalHits = 0;
+ MultiSearcherThread[] msta =
+ new MultiSearcherThread[searchables.length];
+
+ for (int i = 0; i < searchables.length; i++)
+
+ for (int i = 0; i < searchables.length; i++) {
+ try
catch (InterruptedException ie)
{ + ; // TODO: what should we do with this??? + }+ IOException ioe = msta[i].getIOException();
+ if (ioe == null)
else
{ + // if one search produced an IOException, rethrow it + throw ioe; + }+ }
+
+ ScoreDoc[] scoreDocs = new ScoreDoc[hq.size()];
+ for (int i = hq.size() - 1; i >= 0; i--) // put docs in array
+ scoreDocs[i] = (ScoreDoc) hq.pop();
+
+ return new TopDocs(totalHits, scoreDocs);
+ }
+
+ /** Lower-level search API.
+ *
+ * <p>
is called for every non-
zero
+ * scoring document.
+ *
+ * <p>Applications should only use this if they need <i>all</i> of the
+ * matching documents. The high-level search API (
) is usually more efficient, as it skips
+ * non-high-scoring hits.
+ *
+ * @param query to match documents
+ * @param filter if non-null, a bitset used to eliminate some documents
+ * @param results to receive hits
+ *
+ * TODO: parallelize this one too
+ */
+ public void search(Query query, Filter filter, final HitCollector
results)
+ throws IOException {
+ for (int i = 0; i < searchables.length; i++) {
+
+ final int start = starts[i];
+
+ searchables[i].search(query, filter, new HitCollector()
{
+ public void collect(int doc, float score)
+ });
+
+ }
+ }
+
+ /*
+ * TODO: this one could be parallelized too
+ * @see
org.apache.lucene.search.Searchable#rewrite(org.apache.lucene.search.Query)
+ */
+ public Query rewrite(Query original) throws IOException {
+ Query[] queries = new Query[searchables.length];
+ for (int i = 0; i < searchables.length; i++)
+ return original.combine(queries);
+ }
+
+}
+
+/**
+ * A thread subclass for searching a single searchable
+ */
+class MultiSearcherThread extends Thread {
+
+ private Searchable searchable;
+ private Query query;
+ private Filter filter;
+ private int nDocs;
+ private int hits;
+ private TopDocs docs;
+ private int i;
+ private HitQueue hq;
+ private int[] starts;
+ private IOException ioe;
+
+ public MultiSearcherThread(
+ Searchable searchable,
+ Query query,
+ Filter filter,
+ int nDocs,
+ HitQueue hq,
+ int i,
+ int[] starts,
+ String name)
+
+ public void run() {
+ try
+ // Store the IOException for later use by the caller of this
thread
+ catch (IOException ioe)
+ if (ioe == null) {
+ 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)
// no more scores > minScore
+ }
+ }
+ }
+
+ public int hits()
+
+ public IOException getIOException()
+
+}
Index: test/org/apache/lucene/search/TestMultiSearcher.java
===================================================================
RCS file:
/home/cvspublic/jakarta-lucene/src/test/org/apache/lucene/search/TestMultiSe
archer.java,v
retrieving revision 1.4
diff -u -r1.4 TestMultiSearcher.java
— test/org/apache/lucene/search/TestMultiSearcher.java 26 Nov 2002
17:31:43 -0000 1.4
+++ test/org/apache/lucene/search/TestMultiSearcher.java 8 Jan 2004
18:00:02 -0000
@@ -81,6 +81,14 @@
super(name);
}
+ /**
+ * Return a new instance of the concrete MultiSearcher class
+ * used in this test
+ */
+ protected MultiSearcher getMultiSearcherInstance(Searcher[] searchers)
throws IOException
+
public void testEmptyIndex()
throws Exception
{
@@ -134,7 +142,7 @@
searchers[0] = new IndexSearcher(indexStoreB);
searchers[1] = new IndexSearcher(indexStoreA);
// creating the multiSearcher
- Searcher mSearcher = new MultiSearcher(searchers);
+ Searcher mSearcher = getMultiSearcherInstance(searchers);
// performing the search
Hits hits = mSearcher.search(query);
@@ -171,7 +179,7 @@
searchers2[0] = new IndexSearcher(indexStoreB);
searchers2[1] = new IndexSearcher(indexStoreA);
// creating the mulitSearcher
- Searcher mSearcher2 = new MultiSearcher(searchers2);
+ Searcher mSearcher2 = getMultiSearcherInstance(searchers2);
// performing the same search
Hits hits2 = mSearcher2.search(query);
@@ -213,7 +221,7 @@
searchers3[0] = new IndexSearcher(indexStoreB);
searchers3[1] = new IndexSearcher(indexStoreA);
// creating the mulitSearcher
- Searcher mSearcher3 = new MultiSearcher(searchers3);
+ Searcher mSearcher3 = getMultiSearcherInstance(searchers3);
// performing the same search
Hits hits3 = mSearcher3.search(query);
Index: test/org/apache/lucene/search/TestParallelMultiSearcher.java
===================================================================
RCS file: test/org/apache/lucene/search/TestParallelMultiSearcher.java
diff -N test/org/apache/lucene/search/TestParallelMultiSearcher.java
— /dev/null 1 Jan 1970 00:00:00 -0000
+++ test/org/apache/lucene/search/TestParallelMultiSearcher.java 8 Jan
2004
18:00:02 -0000
@@ -0,0 +1,73 @@
+package org.apache.lucene.search;
+
+/* ====================================================================
+ * The Apache Software License, Version 1.1
+ *
+ * Copyright (c) 2001 The Apache Software Foundation. All rights
+ * reserved.
+ *
+ * Redistribution and use in source and binary forms, with or without
+ * modification, are permitted provided that the following conditions
+ * are met:
+ *
+ * 1. Redistributions of source code must retain the above copyright
+ * notice, this list of conditions and the following disclaimer.
+ *
+ * 2. Redistributions in binary form must reproduce the above copyright
+ * notice, this list of conditions and the following disclaimer in
+ * the documentation and/or other materials provided with the
+ * distribution.
+ *
+ * 3. The end-user documentation included with the redistribution,
+ * if any, must include the following acknowledgment:
+ * "This product includes software developed by the
+ * Apache Software Foundation (http://www.apache.org/)."
+ * Alternately, this acknowledgment may appear in the software itself,
+ * if and wherever such third-party acknowledgments normally appear.
+ *
+ * 4. The names "Apache" and "Apache Software Foundation" and
+ * "Apache Lucene" must not be used to endorse or promote products
+ * derived from this software without prior written permission. For
+ * written permission, please contact apache@apache.org.
+ *
+ * 5. Products derived from this software may not be called "Apache",
+ * "Apache Lucene", nor may "Apache" appear in their name, without
+ * prior written permission of the Apache Software Foundation.
+ *
+ * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
+ * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
+ * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+ * DISCLAIMED. IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
+ * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+ * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+ * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
+ * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
+ * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
+ * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
+ * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
+ * SUCH DAMAGE.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation. For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ */
+
+import java.io.IOException;
+
+/**
+ * Unit tests for the ParallelMultiSearcher
+ */
+public class TestParallelMultiSearcher extends TestMultiSearcher {
+
+ public TestParallelMultiSearcher(String name)
+
+ protected MultiSearcher getMultiSearcherInstance(Searcher[] searchers)
+ throws IOException
+
+}