Return-Path: Delivered-To: apmail-lucene-java-user-archive@www.apache.org Received: (qmail 26817 invoked from network); 17 Mar 2006 13:59:11 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (209.237.227.199) by minotaur.apache.org with SMTP; 17 Mar 2006 13:59:11 -0000 Received: (qmail 14535 invoked by uid 500); 17 Mar 2006 13:59:04 -0000 Delivered-To: apmail-lucene-java-user-archive@lucene.apache.org Received: (qmail 14508 invoked by uid 500); 17 Mar 2006 13:59:04 -0000 Mailing-List: contact java-user-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: java-user@lucene.apache.org Delivered-To: mailing list java-user@lucene.apache.org Received: (qmail 14497 invoked by uid 99); 17 Mar 2006 13:59:03 -0000 Received: from asf.osuosl.org (HELO asf.osuosl.org) (140.211.166.49) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 17 Mar 2006 05:59:03 -0800 X-ASF-Spam-Status: No, hits=0.0 required=10.0 tests=HTML_MESSAGE,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (asf.osuosl.org: domain of peterlkeegan@gmail.com designates 64.233.184.204 as permitted sender) Received: from [64.233.184.204] (HELO wproxy.gmail.com) (64.233.184.204) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 17 Mar 2006 05:59:02 -0800 Received: by wproxy.gmail.com with SMTP id i31so580603wra for ; Fri, 17 Mar 2006 05:58:41 -0800 (PST) DomainKey-Signature: a=rsa-sha1; q=dns; c=nofws; s=beta; d=gmail.com; h=received:message-id:date:from:to:subject:in-reply-to:mime-version:content-type:references; b=F8jny/QhvVStNXtlBWfW8RdWQPTLWLw/YDjOLZygGodQV7PmBq07dlsEgKph6zQc9A8ymd3r8CehQvqxET1ageuzEELQ+2/Sr0peOjvI0stOyYhdclZbdiKdvajn1aJ9oXEkRiIL1h87vhU/XTDp2oGvdXHT1HMnOEoH3fa861Q= Received: by 10.54.86.2 with SMTP id j2mr1530057wrb; Fri, 17 Mar 2006 05:58:41 -0800 (PST) Received: by 10.54.113.3 with HTTP; Fri, 17 Mar 2006 05:58:41 -0800 (PST) Message-ID: Date: Fri, 17 Mar 2006 08:58:41 -0500 From: "Peter Keegan" To: java-user@lucene.apache.org Subject: Re: Throughput doesn't increase when using more concurrent threads In-Reply-To: MIME-Version: 1.0 Content-Type: multipart/alternative; boundary="----=_Part_1186_22142123.1142603921153" References: <440DD3A6.2020600@apache.org> <88c6a6720603101400u39d5acf9n20fa89fcc8cdc8e2@mail.gmail.com> X-Virus-Checked: Checked by ClamAV on apache.org X-Spam-Rating: minotaur.apache.org 1.6.2 0/1000/N ------=_Part_1186_22142123.1142603921153 Content-Type: text/plain; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable Content-Disposition: inline I did some additional testing with Chris's patch and mine (based on Doug's note) vs. no patch and found that all 3 produced the same throughput - abou= t 330 qps - over a longer period. So, there seems to be a point of diminishin= g returns to adding more cpus. The dual core Opterons (8 cpu) still win handily at 400 qps. Peter On 3/13/06, Peter Keegan wrote: > > Chris, > My apologies - this error was apparently caused by a file format mismatch > (probably line endings). > Thanks, > Peter > > > On 3/13/06, Peter Keegan wrote: > > > > Chris, > > > > Should this patch work against the current code base? I'm getting this > > error: > > > > D:\lucene-1.9>patch -b -p0 -i nio-lucene-1.9.patch > > patching file src/java/org/apache/lucene/index/CompoundFileReader.java > > patching file src/java/org/apache/lucene/index/FieldsReader.java > > missing header for unified diff at line 45 of patch > > can't find file to patch at input line 45 > > Perhaps you used the wrong -p or --strip option? > > The text leading up to this was: > > -------------------------- > > | +47,9 @@ > > | fieldsStream =3D d.openInput(segment + ".fdt"); > > | indexStream =3D d.openInput(segment + ".fdx"); > > | > > |+ fstream =3D new ThreadStream(fieldsStream); > > |+ istream =3D new ThreadStream(indexStream); > > |+ > > | size =3D (int)(indexStream.length() / 8); > > | } > > | > > -------------------------- > > > > Thanks, > > Peter > > > > > > > > On 3/10/06, Chris Lamprecht wrote: > > > > > > Peter, > > > > > > I think this is similar to the patch in this bugzilla task: > > > > > > http://issues.apache.org/bugzilla/show_bug.cgi?id=3D35838 > > > the patch itself is > > > http://issues.apache.org/bugzilla/attachment.cgi?id=3D15757 > > > > > > (BTW does JIRA have a way to display the patch diffs?) > > > > > > The above patch also has a change to SegmentReader to avoid > > > synchronization on isDeleted(). However, with that patch, you no > > > longer have the guarantee that one thread will immediately see > > > deletions by another thread. This was fine for my purposes, and > > > resulted in a big performance boost when there were deleted documents= , > > > > > > but it may not be "correct" for others' needs. > > > > > > -chris > > > On 3/10/06, Peter Keegan wrote: > > > > > 3. Use the ThreadLocal's FieldReader in the document() method. > > > > > > > > As I understand it, this means that the document method no longer > > > needs to > > > > be synchronized, right? > > > > > > > > I've made these changes and it does appear to improve performance. > > > Random > > > > snapshots of the stack traces show only an occasional lock in > > > 'isDeleted'. > > > > Mostly, though, the threads are busy scoring and adding results to > > > priority > > > > queues, which is great. I've included some sample stacks, below. > > > I'll report > > > > the new query rates after it has run for at least overnight, and I'= d > > > be > > > > happy submit these changes to the lucene committers, if interested. > > > > > > > > Peter > > > > > > > > > > > > Sample stack traces: > > > > > > > > "QueryThread group 1,#8" prio=3D1 tid=3D0x0000002ce48eeb80 nid=3D0x= 6b87 > > > runnable > > > > [0x0000000043887000..0x0000000043887bb0] > > > > at org.apache.lucene.search.FieldSortedHitQueue.lessThan( > > > > FieldSortedHitQueue.java:108) > > > > at org.apache.lucene.util.PriorityQueue.insert( > > > PriorityQueue.java :61) > > > > at org.apache.lucene.search.FieldSortedHitQueue.insert( > > > > FieldSortedHitQueue.java:85) > > > > at org.apache.lucene.search.FieldSortedHitQueue.insert( > > > > FieldSortedHitQueue.java:92) > > > > at org.apache.lucene.search.TopFieldDocCollector.collect( > > > > TopFieldDocCollector.java:51) > > > > at org.apache.lucene.search.TermScorer.score(TermScorer.java:75= ) > > > > at org.apache.lucene.search.TermScorer.score (TermScorer.java > > > :60) > > > > at org.apache.lucene.search.IndexSearcher.search( > > > IndexSearcher.java:132) > > > > at org.apache.lucene.search.IndexSearcher.search( > > > IndexSearcher.java:110) > > > > at org.apache.lucene.search.MultiSearcher.search ( > > > MultiSearcher.java:225) > > > > at org.apache.lucene.search.Hits.getMoreDocs(Hits.java:65) > > > > at org.apache.lucene.search.Hits.(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search (Searcher.java:62) > > > > > > > > "QueryThread group 1,#5" prio=3D1 tid=3D0x0000002ce4d659f0 nid=3D0x= 6b84 > > > runnable > > > > [0x0000000043584000..0x0000000043584d30] > > > > at org.apache.lucene.search.TermScorer.score (TermScorer.java > > > :75) > > > > at org.apache.lucene.search.TermScorer.score(TermScorer.java:60= ) > > > > at org.apache.lucene.search.IndexSearcher.search( > > > IndexSearcher.java:132) > > > > at org.apache.lucene.search.IndexSearcher.search ( > > > IndexSearcher.java:110) > > > > at org.apache.lucene.search.MultiSearcher.search( > > > MultiSearcher.java:225) > > > > at org.apache.lucene.search.Hits.getMoreDocs(Hits.java:65) > > > > at org.apache.lucene.search.Hits .(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search(Searcher.java:62) > > > > > > > > "QueryThread group 1,#4" prio=3D1 tid=3D0x0000002ce10afd50 nid=3D0x= 6b83 > > > runnable > > > > [0x0000000043483000..0x0000000043483db0] > > > > at org.apache.lucene.store.MMapDirectory$MMapIndexInput.readByt= e > > > ( > > > > MMapDirectory.java:46) > > > > at org.apache.lucene.store.IndexInput.readVInt(IndexInput.java > > > :56) > > > > at org.apache.lucene.index.SegmentTermDocs.next ( > > > SegmentTermDocs.java > > > > :101) > > > > at org.apache.lucene.index.SegmentTermDocs.skipTo( > > > SegmentTermDocs.java > > > > :194) > > > > at org.apache.lucene.search.TermScorer.skipTo(TermScorer.java > > > :144) > > > > at org.apache.lucene.search.ConjunctionScorer.doNext( > > > > ConjunctionScorer.java:56) > > > > at org.apache.lucene.search.ConjunctionScorer.next( > > > > ConjunctionScorer.java:51) > > > > at org.apache.lucene.search.BooleanScorer2.score ( > > > BooleanScorer2.java > > > > :290) > > > > at org.apache.lucene.search.IndexSearcher.search( > > > IndexSearcher.java:132) > > > > at org.apache.lucene.search.IndexSearcher.search( > > > IndexSearcher.java:110) > > > > at org.apache.lucene.search.MultiSearcher.search ( > > > MultiSearcher.java:225) > > > > at org.apache.lucene.search.Hits.getMoreDocs(Hits.java:65) > > > > at org.apache.lucene.search.Hits.(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search (Searcher.java:62) > > > > > > > > "QueryThread group 1,#3" prio=3D1 tid=3D0x0000002ce48959f0 nid=3D0x= 6b82 > > > runnable > > > > [0x0000000043382000..0x0000000043382e30] > > > > at java.util.LinkedList.listIterator(LinkedList.java :523) > > > > at java.util.AbstractList.listIterator(AbstractList.java:349) > > > > at java.util.AbstractSequentialList.iterator( > > > AbstractSequentialList.java > > > > :250) > > > > at org.apache.lucene.search.ConjunctionScorer.score ( > > > > ConjunctionScorer.java:80) > > > > at org.apache.lucene.search.BooleanScorer2$2.score( > > > BooleanScorer2.java > > > > :186) > > > > at org.apache.lucene.search.BooleanScorer2.score( > > > BooleanScorer2.java > > > > :327) > > > > at org.apache.lucene.search.BooleanScorer2.score( > > > BooleanScorer2.java > > > > :291) > > > > at org.apache.lucene.search.IndexSearcher.search( > > > IndexSearcher.java:132) > > > > at org.apache.lucene.search.IndexSearcher.search ( > > > IndexSearcher.java:110) > > > > at org.apache.lucene.search.MultiSearcher.search( > > > MultiSearcher.java:225) > > > > at org.apache.lucene.search.Hits.getMoreDocs(Hits.java:65) > > > > at org.apache.lucene.search.Hits .(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search(Searcher.java:62) > > > > > > > > > > > > On 3/7/06, Doug Cutting < cutting@apache.org> wrote: > > > > > > > > > > Peter Keegan wrote: > > > > > > I ran a query performance tester against 8-cpu and 16-cpu Xeon > > > servers > > > > > > (16/32 cpu hyperthreaded). on Linux. Here are the results: > > > > > > > > > > > > 8-cpu: 275 qps > > > > > > 16-cpu: 305 qps > > > > > > (the dual-core Opteron servers are still faster) > > > > > > > > > > > > Here is the stack trace of 8 of the 16 query threads during the > > > test: > > > > > > > > > > > > at org.apache.lucene.index.SegmentReader.document( > > > > > SegmentReader.java > > > > > > :281) > > > > > > - waiting to lock <0x0000002adf5b2110> (a > > > > > > org.apache.lucene.index.SegmentReader) > > > > > > at org.apache.lucene.search.IndexSearcher.doc( > > > IndexSearcher.java > > > > > :83) > > > > > > at org.apache.lucene.search.MultiSearcher.doc ( > > > MultiSearcher.java > > > > > > :146) > > > > > > at org.apache.lucene.search.Hits.doc(Hits.java:103) > > > > > > > > > > > > SegmentReader.document is a synchronized method. I have one > > > stored field > > > > > > (binary, uncompressed) with and average length of 0.5Kb. The > > > retrieval > > > > > of > > > > > > this stored field is within this synchronized code. Since I am > > > using > > > > > > MMapDirectory, does this retrieval need to be synchronized? > > > > > > > > > > Yes, since in FieldReader the file positions must be synchronized= . > > > > > > > > > > The way to avoid this would be to: > > > > > > > > > > 1. Add a clone() method to FieldReader that clones it's two > > > IndexInputs. > > > > > 2. Add a ThreadLocal to SegmentReader whose value is a cloned > > > FieldReader. > > > > > 3. Use the ThreadLocal's FieldReader in the document() method. > > > > > > > > > > TermInfosReader has a similar optimization, using a ThreadLocal > > > > > containing a SegmentTermEnum for each thread. > > > > > > > > > > Doug > > > > > > > > > > > > > --------------------------------------------------------------------- > > > > > To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org > > > > > For additional commands, e-mail: java-user-help@lucene.apache.org > > > > > > > > > > > > > > > > > > > > > > > > --------------------------------------------------------------------- > > > To unsubscribe, e-mail: java-user-unsubscribe@lucene.apache.org > > > For additional commands, e-mail: java-user-help@lucene.apache.org > > > > > > > > > ------=_Part_1186_22142123.1142603921153--