the list of IDs does change relatively frequently, but this doesn't seem to
have very much impact on the performance of the query as far as I can tell.
attached are the stacks
thanks,
steve
On Wed, Jul 31, 2013 at 6:33 AM, Mikhail Khludnev <
[email protected]> wrote:
> On Wed, Jul 31, 2013 at 1:10 AM, Steven Bower <[email protected]> wrote:
>
> >
> > not sure what you mean by good hit raitio?
> >
>
> I mean such queries are really expensive (even on cache hit), so if the
> list of ids changes every time, it never hit cache and hence executes these
> heavy queries every time. It's well known performance problem.
>
>
> > Here are the stacks...
> >
> they seems like hotspots, and shows index reading that's reasonable. But I
> can't see what caused these readings, to get that I need whole stack of hot
> thread.
>
>
> >
> > Name Time (ms) Own Time (ms)
> >
> >
> org.apache.lucene.search.MultiTermQueryWrapperFilter.getDocIdSet(AtomicReaderContext,
> > Bits) 300879 203478
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader$BlockDocsEnum.nextDoc()
> > 45539 19
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader$BlockDocsEnum.refillDocs()
> > 45519 40
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader.readVIntBlock(IndexInput,
> > int[], int[], int, boolean) 24352 0
> > org.apache.lucene.store.DataInput.readVInt() 24352 24352
> > org.apache.lucene.codecs.lucene41.ForUtil.readBlock(IndexInput, byte[],
> > int[]) 21126 14976
> > org.apache.lucene.store.ByteBufferIndexInput.readBytes(byte[], int, int)
> > 6150 0 java.nio.DirectByteBuffer.get(byte[], int, int)
> > 6150 0
> > java.nio.Bits.copyToArray(long, Object, long, long, long) 6150 6150
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum.docs(Bits,
> > DocsEnum, int) 35342 421
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.decodeMetaData()
> > 34920 27939
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader.nextTerm(FieldInfo,
> > BlockTermState) 6980 6980
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum.next()
> > 14129 1053
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.loadNextFloorBlock()
> > 5948 261
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.loadBlock()
> > 5686 199
> > org.apache.lucene.store.ByteBufferIndexInput.readBytes(byte[], int, int)
> > 3606 0 java.nio.DirectByteBuffer.get(byte[], int, int)
> > 3606 0
> > java.nio.Bits.copyToArray(long, Object, long, long, long) 3606 3606
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader.readTermsBlock(IndexInput,
> > FieldInfo, BlockTermState) 1879 80
> > org.apache.lucene.store.ByteBufferIndexInput.readBytes(byte[], int, int)
> > 1798 0 java.nio.DirectByteBuffer.get(byte[], int, int)
> > 1798 0
> > java.nio.Bits.copyToArray(long, Object, long, long, long) 1798 1798
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.next()
> > 4010 3324
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.nextNonLeaf()
> > 685 685
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.loadBlock()
> > 3117 144
> > org.apache.lucene.store.ByteBufferIndexInput.readBytes(byte[], int, int)
> > 1861 0 java.nio.DirectByteBuffer.get(byte[], int, int) 1861
> > 0
> > java.nio.Bits.copyToArray(long, Object, long, long, long) 1861 1861
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader.readTermsBlock(IndexInput,
> > FieldInfo, BlockTermState) 1090 19
> > org.apache.lucene.store.ByteBufferIndexInput.readBytes(byte[], int, int)
> > 1070 0 java.nio.DirectByteBuffer.get(byte[], int, int)
> > 1070 0
> > java.nio.Bits.copyToArray(long, Object, long, long, long) 1070 1070
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum.initIndexInput()
> > 20 0 org.apache.lucene.store.ByteBufferIndexInput.clone()
> > 20 0
> > org.apache.lucene.store.ByteBufferIndexInput.clone() 20 0
> > org.apache.lucene.store.ByteBufferIndexInput.buildSlice(long, long) 20
> > 0
> > org.apache.lucene.util.WeakIdentityMap.put(Object, Object) 20 0
> >
> org.apache.lucene.util.WeakIdentityMap$IdentityWeakReference.<init>(Object,
> > ReferenceQueue) 20 0
> > java.lang.System.identityHashCode(Object) 20 20
> > org.apache.lucene.index.FilteredTermsEnum.docs(Bits, DocsEnum, int)
> > 1485 527
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum.docs(Bits,
> > DocsEnum, int) 957 0
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.decodeMetaData()
> > 957 513
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader.nextTerm(FieldInfo,
> > BlockTermState) 443 443
> > org.apache.lucene.index.FilteredTermsEnum.next() 874 324
> >
> >
> org.apache.lucene.search.NumericRangeQuery$NumericRangeTermsEnum.accept(BytesRef)
> > 368 0
> >
> >
> org.apache.lucene.util.BytesRef$UTF8SortedAsUnicodeComparator.compare(Object,
> > Object) 368 368
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum.next()
> > 160 0
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.loadNextFloorBlock()
> > 160 0
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.loadBlock()
> > 160 0
> > org.apache.lucene.store.ByteBufferIndexInput.readBytes(byte[], int, int)
> > 120
> > 0
> >
> >
> org.apache.lucene.codecs.lucene41.Lucene41PostingsReader.readTermsBlock(IndexInput,
> > FieldInfo, BlockTermState) 39 0
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum.seekCeil(BytesRef,
> > boolean) 19 0
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum$Frame.loadBlock()
> > 19 0
> >
> >
> org.apache.lucene.codecs.BlockTreeTermsReader$FieldReader$SegmentTermsEnum.initIndexInput()
> > 19 0 org.apache.lucene.store.ByteBufferIndexInput.clone()
> > 19 0
> > org.apache.lucene.store.ByteBufferIndexInput.clone() 19 0
> > org.apache.lucene.store.ByteBufferIndexInput.buildSlice(long, long) 19
> > 0
> > org.apache.lucene.util.WeakIdentityMap.put(Object, Object) 19 0
> >
> org.apache.lucene.util.WeakIdentityMap$IdentityWeakReference.<init>(Object,
> > ReferenceQueue) 19 0
> > java.lang.System.identityHashCode(Object) 19 19
> > org.apache.lucene.util.FixedBitSet.<init>(int) 28 28
> >
> >
> > On Tue, Jul 30, 2013 at 4:18 PM, Mikhail Khludnev <
> > [email protected]> wrote:
> >
> > > On Tue, Jul 30, 2013 at 12:45 AM, Steven Bower <[email protected]
> > > >wrote:
> > >
> > > >
> > > > - Most of my time (98%) is being spent in
> > > > java.nio.Bits.copyToByteArray(long,Object,long,long) which is being
> > >
> > >
> > > Steven, please
> > >
> >
> http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html.my
> > > benchmarking experience shows that NIO is a turtle, absolutely.
> > >
> > > also, are you sure that fq=(vid:86XXX73 OR vid:86XXX20 ..... has good
> hit
> > > ratio? otherwise it's a well known beast.
> > >
> > > could you also show deeper stack, to make sure what causes to excessive
> > > reading?
> > >
> > >
> > >
> > > --
> > > Sincerely yours
> > > Mikhail Khludnev
> > > Principal Engineer,
> > > Grid Dynamics
> > >
> > > <http://www.griddynamics.com>
> > > <[email protected]>
> > >
> >
>
>
>
> --
> Sincerely yours
> Mikhail Khludnev
> Principal Engineer,
> Grid Dynamics
>
> <http://www.griddynamics.com>
> <[email protected]>
>
Back traces
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------+
|
Name
| Time (ms) |
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------+
|
+---org.apache.lucene.search.MultiTermQueryWrapperFilter.getDocIdSet(AtomicReaderContext,
Bits) |
300,879 100 % |
| |
| |
|
+---org.apache.lucene.search.ConstantScoreQuery$ConstantWeight.scorer(AtomicReaderContext,
boolean, boolean, Bits) |
|
| |
| |
|
+---org.apache.lucene.search.BooleanQuery$BooleanWeight.scorer(AtomicReaderContext,
boolean, boolean, Bits) |
300,580 99 % |
| | |
| |
| | +---org.apache.lucene.search.IndexSearcher.search(List, Weight,
Collector)
| 207,673 69 % |
| | | |
| |
| | | +---org.apache.lucene.search.IndexSearcher.search(Query, Filter,
Collector)
| |
| | | |
| |
| | |
+---org.apache.solr.search.SolrIndexSearcher.getDocListNC(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) |
|
| | | |
| |
| | |
+---org.apache.solr.search.SolrIndexSearcher.getDocListC(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) |
|
| | | |
| |
| | |
+---org.apache.solr.search.SolrIndexSearcher.search(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) |
|
| | | |
| |
| | |
+---org.apache.solr.handler.component.QueryComponent.process(ResponseBuilder)
|
|
| | | |
| |
| | |
+---org.apache.solr.handler.component.SearchHandler.handleRequestBody(SolrQueryRequest,
SolrQueryResponse) | |
| | | |
| |
| | |
+---org.apache.solr.handler.RequestHandlerBase.handleRequest(SolrQueryRequest,
SolrQueryResponse) |
|
| | | |
| |
| | |
+---org.apache.solr.core.SolrCore.execute(SolrRequestHandler, SolrQueryRequest,
SolrQueryResponse) | |
| | | |
| |
| | |
+---org.apache.solr.servlet.SolrDispatchFilter.execute(HttpServletRequest,
SolrRequestHandler, SolrQueryRequest, SolrQueryResponse) |
|
| | | |
| |
| | |
+---org.apache.solr.servlet.SolrDispatchFilter.doFilter(ServletRequest,
ServletResponse, FilterChain, boolean) |
|
| | | |
| |
| | |
+---org.apache.solr.servlet.SolrDispatchFilter.doFilter(ServletRequest,
ServletResponse, FilterChain) |
|
| | |
| |
| | +---org.apache.solr.search.FilterImpl$FilterSet.iterator()
| 92,906 31 % |
| | |
| |
| |
+---org.apache.lucene.search.ConstantScoreQuery$ConstantWeight.scorer(AtomicReaderContext,
boolean, boolean, Bits) |
|
| | |
| |
| | +---org.apache.lucene.search.IndexSearcher.search(List, Weight,
Collector)
| |
| | |
| |
| | +---org.apache.lucene.search.IndexSearcher.search(Query, Filter,
Collector)
| |
| | |
| |
| |
+---org.apache.solr.search.SolrIndexSearcher.getDocListNC(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) | |
| | |
| |
| |
+---org.apache.solr.search.SolrIndexSearcher.getDocListC(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) | |
| | |
| |
| |
+---org.apache.solr.search.SolrIndexSearcher.search(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) |
|
| | |
| |
| |
+---org.apache.solr.handler.component.QueryComponent.process(ResponseBuilder)
|
|
| | |
| |
| |
+---org.apache.solr.handler.component.SearchHandler.handleRequestBody(SolrQueryRequest,
SolrQueryResponse) | |
| | |
| |
| |
+---org.apache.solr.handler.RequestHandlerBase.handleRequest(SolrQueryRequest,
SolrQueryResponse) | |
| | |
| |
| |
+---org.apache.solr.core.SolrCore.execute(SolrRequestHandler, SolrQueryRequest,
SolrQueryResponse) | |
| | |
| |
| |
+---org.apache.solr.servlet.SolrDispatchFilter.execute(HttpServletRequest,
SolrRequestHandler, SolrQueryRequest, SolrQueryResponse) | |
| |
| |
| +---org.apache.solr.search.FilterImpl$FilterSet.iterator()
| 298 0 % |
| |
| |
|
+---org.apache.lucene.search.ConstantScoreQuery$ConstantWeight.scorer(AtomicReaderContext,
boolean, boolean, Bits) |
|
| |
| |
| +---org.apache.lucene.search.IndexSearcher.search(List, Weight,
Collector)
| |
| |
| |
| +---org.apache.lucene.search.IndexSearcher.search(Query, Filter,
Collector)
| |
| |
| |
|
+---org.apache.solr.search.SolrIndexSearcher.getDocListNC(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) |
|
| |
| |
|
+---org.apache.solr.search.SolrIndexSearcher.getDocListC(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) | |
| |
| |
|
+---org.apache.solr.search.SolrIndexSearcher.search(SolrIndexSearcher$QueryResult,
SolrIndexSearcher$QueryCommand) |
|
| |
| |
|
+---org.apache.solr.handler.component.QueryComponent.process(ResponseBuilder)
|
|
| |
| |
|
+---org.apache.solr.handler.component.SearchHandler.handleRequestBody(SolrQueryRequest,
SolrQueryResponse) | |
| |
| |
|
+---org.apache.solr.handler.RequestHandlerBase.handleRequest(SolrQueryRequest,
SolrQueryResponse) | |
| |
| |
|
+---org.apache.solr.core.SolrCore.execute(SolrRequestHandler, SolrQueryRequest,
SolrQueryResponse) | |
| |
| |
|
+---org.apache.solr.servlet.SolrDispatchFilter.execute(HttpServletRequest,
SolrRequestHandler, SolrQueryRequest, SolrQueryResponse) | |
+-------------------------------------------------------------------------------------------------------------------------------------------------------------------+------------------+
Generated by YourKit Java Profiler 12.0.5 Jul 31, 2013 4:31:48 PM