Opening your IndexReader with readOnly=true should also fix it, I think. Mike
On Thu, Aug 6, 2009 at 8:41 AM, Carl Austin<carl.aus...@detica.com> wrote: > Thanks Mike, > > Running this with the 2.9 build does resolve the issue it would seem. > > Unfortunately I can't move to 2.9, especially as it isn't in release yet. Is > there a work-around for 2.4 known that will allow me to get around this issue > as I notice the patches change some underlying classes such as > MultiSegmentReader? What I am doing does regularly have the chance to call > this query concurrently multiple times, which cause quite an issue. > > Thanks > > Carl > > -----Original Message----- > From: Michael McCandless [mailto:luc...@mikemccandless.com] > Sent: 06 August 2009 13:26 > To: java-user@lucene.apache.org > Subject: Re: MatchAllDocsQuery concurrency issue > > Most likely you're hitting this issue: > > https://issues.apache.org/jira/browse/LUCENE-1316 > > Which is fixed in 2.9. Can you try running with 2.9 to confirm? > > Mike > > On Thu, Aug 6, 2009 at 8:19 AM, Carl Austin<carl.aus...@detica.com> wrote: >> Hi, >> >> I have been seeing an issue running MatchAllDocsQueries concurrently. >> Running one against a test index is very fast (70 ms). Running two >> concurrently can take 5-25 seconds on the same test index! This issue >> doesn't occur with any other type of query I have used. >> Because of this, I have put together a few test classes to demonstrate >> this in a simple manner. These recreate the issue with only the most >> basic Lucene api usage. >> >> I have incldued the code for the classes below in this email. Just run >> TestIndexCreator with a single argument of where to create the test >> index. This will create an index with 999999 documents in, each >> document having only 1 field with the value "testing" in that field. >> Only one unique term for the whole index. >> Then run the class Test with the same argument as the index creator. >> This will run a 4 tests with a 5 second wait between each. The first >> runs a single term query for "testing". The second a single match all >> docs query. The third 2 concurrent term queries and the fourth 2 >> concurrent match all docs queries. It is the fourth that takes a >> relatively long time to complete. >> On a larger index I have here it can take 8 minutes to run just 2 >> match all docs queries concurrently. >> >> I look forward to hearing further on this. I can provide this in Jar >> for if wished, though I don't know if you can attach to these emails. >> >> Thanks >> >> Carl >> >> >> Source Code: >> >> import org.apache.lucene.analysis.KeywordAnalyzer; >> import org.apache.lucene.document.Document; >> import org.apache.lucene.document.Field; import >> org.apache.lucene.document.Field.Index; >> import org.apache.lucene.document.Field.Store; >> import org.apache.lucene.index.IndexReader; >> import org.apache.lucene.index.IndexWriter; >> >> >> public class TestIndexCreator { >> >> >> public static void main(String[] args) throws Exception >> { >> if (!IndexReader.indexExists(args[0])) >> { >> IndexWriter w = new IndexWriter(args[0], new >> KeywordAnalyzer(), true); >> for(int i =0; i < 1000000; i++) >> { >> Document doc = new Document(); >> doc.add(new Field("testfield", >> "testing", Store.YES, Index.ANALYZED)); >> w.addDocument(doc); >> if (i % 10000 == 0) >> { >> System.out.println(i); >> } >> } >> w.optimize(); >> w.close(); >> } >> } >> >> } >> >> import java.io.IOException; >> >> import org.apache.lucene.index.IndexReader; >> import org.apache.lucene.index.Term; >> import org.apache.lucene.search.IndexSearcher; >> import org.apache.lucene.search.MatchAllDocsQuery; >> import org.apache.lucene.search.Query; import >> org.apache.lucene.search.TermQuery; >> import org.apache.lucene.search.TopDocCollector; >> >> class TestThread implements Runnable >> { >> private IndexReader r; >> private int x; >> private Test callback; >> private boolean termSearches; >> >> public TestThread(IndexReader r, int x, Test callback, >> boolean termSearches) >> { >> this.r = r; >> this.x = x; >> this.callback = callback; >> this.termSearches = termSearches; >> } >> >> public void run() { >> long time = System.currentTimeMillis(); >> System.out.println("Thread " + x + " start"); >> IndexSearcher s = new IndexSearcher(r); >> Query q = null; >> if (termSearches) >> { >> q = new TermQuery(new Term("testfield", >> "testing")); >> } >> else >> { >> q = new MatchAllDocsQuery(); >> >> } >> TopDocCollector col = new TopDocCollector(50); >> try { >> s.search(q, col); >> } catch (IOException e) { >> e.printStackTrace(); >> } >> System.out.println("Time taken for " + x + " = " >> + (System.currentTimeMillis() - time)); >> callback.callback(); >> } >> >> } >> >> import java.io.IOException; >> >> import org.apache.lucene.index.CorruptIndexException; >> import org.apache.lucene.index.IndexReader; >> >> >> public class Test { >> >> private IndexReader r; >> private int noSearches = 0; >> >> public static void main(String[] args) >> { >> System.out.println("Running Single TermQuery Search"); >> Test t = new Test(args[0]); >> try { >> t.runSearches(1, true); >> } catch (Exception e) { >> // TODO Auto-generated catch block >> e.printStackTrace(); >> } >> try { >> Thread.sleep(5000); >> } catch (InterruptedException e1) { >> // TODO Auto-generated catch block >> e1.printStackTrace(); >> } >> System.out.println("Running Single MatchAllDocs >> Search"); >> t = new Test(args[0]); >> try { >> t.runSearches(1, false); >> } catch (Exception e) { >> // TODO Auto-generated catch block >> e.printStackTrace(); >> } >> try { >> Thread.sleep(5000); >> } catch (InterruptedException e1) { >> // TODO Auto-generated catch block >> e1.printStackTrace(); >> } >> System.out.println("Running two concurrent TermQuery >> Searches"); >> t = new Test(args[0]); >> try { >> t.runSearches(2, true); >> } catch (Exception e) { >> // TODO Auto-generated catch block >> e.printStackTrace(); >> } >> try { >> Thread.sleep(5000); >> } catch (InterruptedException e1) { >> // TODO Auto-generated catch block >> e1.printStackTrace(); >> } >> System.out.println("Running two concurrent MatchAllDocs >> Searches"); >> t = new Test(args[0]); >> try { >> t.runSearches(2, false); >> } catch (Exception e) { >> // TODO Auto-generated catch block >> e.printStackTrace(); >> } >> >> } >> >> public Test(String index) >> { >> try { >> r = IndexReader.open(index); >> } catch (CorruptIndexException e) { >> // TODO Auto-generated catch block >> e.printStackTrace(); >> } catch (IOException e) { >> // TODO Auto-generated catch block >> e.printStackTrace(); >> } >> } >> >> public void runSearches(int no, boolean termSearches) throws >> Exception >> { >> noSearches = no; >> for (int i = 0; i < no; i++) >> { >> Thread t1 = new Thread(new TestThread(r, i, >> this, termSearches)); >> t1.start(); >> } >> } >> >> public void callback() >> { >> noSearches--; >> if (noSearches == 0) >> { >> try { >> r.close(); >> } catch (IOException e) { >> // TODO Auto-generated catch block >> e.printStackTrace(); >> } >> } >> } >> >> } >> >> >> >> This message should be regarded as confidential. If you have received this >> email in error please notify the sender and destroy it immediately. >> Statements of intent shall only become binding when confirmed in hard copy >> by an authorised signatory. The contents of this email may relate to >> dealings with other companies within the Detica Limited group of companies. >> >> Detica Limited is registered in England under No: 1337451. >> >> Registered offices: Surrey Research Park, Guildford, Surrey, GU2 7YP, >> England. >> >> > > --------------------------------------------------------------------- > To unsubscribe, e-mail: java-user-unsubscr...@lucene.apache.org > For additional commands, e-mail: java-user-h...@lucene.apache.org > > > --------------------------------------------------------------------- > To unsubscribe, e-mail: java-user-unsubscr...@lucene.apache.org > For additional commands, e-mail: java-user-h...@lucene.apache.org > > --------------------------------------------------------------------- To unsubscribe, e-mail: java-user-unsubscr...@lucene.apache.org For additional commands, e-mail: java-user-h...@lucene.apache.org