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 - about 330 qps - over a longer period. So, there seems to be a point of diminishing returns to adding more cpus. The dual core Opterons (8 cpu) still win handily at 400 qps.
Peter On 3/13/06, Peter Keegan <[EMAIL PROTECTED]> 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 <[EMAIL PROTECTED]> 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 = d.openInput(segment + ".fdt"); > > | indexStream = d.openInput(segment + ".fdx"); > > | > > |+ fstream = new ThreadStream(fieldsStream); > > |+ istream = new ThreadStream(indexStream); > > |+ > > | size = (int)(indexStream.length() / 8); > > | } > > | > > -------------------------- > > > > Thanks, > > Peter > > > > > > > > On 3/10/06, Chris Lamprecht <[EMAIL PROTECTED]> wrote: > > > > > > Peter, > > > > > > I think this is similar to the patch in this bugzilla task: > > > > > > http://issues.apache.org/bugzilla/show_bug.cgi?id=35838 > > > the patch itself is > > > http://issues.apache.org/bugzilla/attachment.cgi?id=15757 > > > > > > (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 <[EMAIL PROTECTED] > 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=1 tid=0x0000002ce48eeb80 nid=0x6b87 > > > 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.<init>(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search (Searcher.java:62) > > > > > > > > "QueryThread group 1,#5" prio=1 tid=0x0000002ce4d659f0 nid=0x6b84 > > > 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 .<init>(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search(Searcher.java:62) > > > > > > > > "QueryThread group 1,#4" prio=1 tid=0x0000002ce10afd50 nid=0x6b83 > > > runnable > > > > [0x0000000043483000..0x0000000043483db0] > > > > at org.apache.lucene.store.MMapDirectory$MMapIndexInput.readByte > > > ( > > > > 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.<init>(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search (Searcher.java:62) > > > > > > > > "QueryThread group 1,#3" prio=1 tid=0x0000002ce48959f0 nid=0x6b82 > > > 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 .<init>(Hits.java:52) > > > > at org.apache.lucene.search.Searcher.search(Searcher.java:62) > > > > > > > > > > > > On 3/7/06, Doug Cutting < [EMAIL PROTECTED]> 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: [EMAIL PROTECTED] > > > > > For additional commands, e-mail: [EMAIL PROTECTED] > > > > > > > > > > > > > > > > > > > > > > > > --------------------------------------------------------------------- > > > To unsubscribe, e-mail: [EMAIL PROTECTED] > > > For additional commands, e-mail: [EMAIL PROTECTED] > > > > > > > > >