Vitaly,

Thanks for your comments.

Unfortunately, thread pool task overload is not the problem. When I
extended the IndexSearcher class last night, I had it create one task per
shard (20 tasks) instead of the default which turned out to be somewhere
around 320 (I didn't realize it created quite so many segments when I
recreated my index yesterday!) and performance was still awful as long as
there were multiple segments per shard.

I understand that the fork/join framework doesn't work well with the
standard IndexSearcher's parallel search implementation, but I extended the
class and my implementation avoids the pitfalls as long as there are very
few (preferably one) segments per shard. The blocking in the original
SegmentWorkerCallable is still there, but I set it up so that if one worker
already has the lock, any others that try to lock it will just throw their
results in a separate queue which is polled by the thread that managed to
get the lock. After all the child tasks are complete, the queue is checked
one more time to be safe and then processing continues. There is basically
no waiting in that chunk of code (per the profiler). After doing some tests
with the ForkJoinPool and custom code as well as a standard
ThreadPoolExecutor, I found that the Fork/Join implementation was faster.

The system I'm working on is much more sensitive to response time (latency)
than overall throughput, so we've had to focus on parallelism within
queries rather than across many queries. Batch processing performance is
still important, but we cannot sacrifice quick "online" responses. It would
be much easier to avoid this whole mess, but we cannot meet our performance
requirements without it.

Regards,

Matt


On Oct 2, 2013 3:36 PM, "Vitaly Funstein" <vfunst...@gmail.com> wrote:

> Matt,
>
> I think you are mostly on track with suspecting thread pool task overload
> as the possible culprit here. First, the old school (prior to Java 7)
> ThreadPoolExecutor only accepts a BlockingQueue to use internally for
> worker tasks, instead of a concurrent variant (not sure why). So this
> internal work queue will become a significant point of contention when
> using the pool in a pattern similar to your use case, i.e. submitting lots
> of tasks to the pool as fast as possible.
>
> Second, I am not too familiar with the internals of the fork/join pool
> implementation in 1.7 (if that's what you're using), but from reading the
> daunting javadoc for ForkJoinTask my rough guess is it's not terribly well
> suited for use in IndexSearcher. In particular, one of the possible
> "non-compliant" behaviors is that a mutex lock is taken for each call()
> invocation that operates on an individual leaf slice. This is evident from
> code inspection, and based on this, I am not sure what benefit, if any,
> multi-threaded search over a multi-segment index would provide in general -
> regardless of the choice of thread pool implementation.
>
> I think a better strategy, as mentioned in another thread, is to optimize
> your system for multiple concurrent queries, rather than focusing on
> forcing each query to run across multiple threads/cores. With that
> approach, you could for instance, set up a non-blocking queue like
> ConcurrentLinkedQueue that would contain individual query tasks, then use a
> fixed pool of worker threads to consume the queue in a loop and run them.
> In this scenario, you shouldn't need to pass an ExecutorService instance to
> IndexSearcher.
>
> A strategy like that should provide for better query throughput, regardless
> of whether each shard consists of a single segment or not - provided each
> of the queries is tied to a particular shard and can't search any others.
>
>
>
> On Tue, Oct 1, 2013 at 4:10 PM, Desidero <desid...@gmail.com> wrote:
>
> > Uwe,
> >
> > I was using a bounded thread pool.
> >
> > I don't know if the problem was the task overload or something about the
> > actual efficiency of searching a single segment rather than iterating
> over
> > multiple AtomicReaderContexts, but I'd lean toward task overload. I will
> do
> > some testing tonight to find out for sure.
> >
> > Matt
> >  Hi,
> >
> > use a bounded thread pool.
> >
> > Uwe
> >
> > -----
> > Uwe Schindler
> > H.-H.-Meier-Allee 63, D-28213 Bremen
> > http://www.thetaphi.de
> > eMail: u...@thetaphi.de
> >
> >
> > > -----Original Message-----
> > > From: Desidero [mailto:desid...@gmail.com]
> > > Sent: Tuesday, October 01, 2013 11:37 PM
> > > To: java-user@lucene.apache.org
> > > Subject: Re: Query performance in Lucene 4.x
> > >
> > > For anyone who was wondering, this was actually resolved in a different
> > > thread today. I misread the information in the
> > > IndexSearcher(IndexReader,ExecutorService) constructor documentation -
> I
> > > was under the impression that it was submitting a thread for each index
> > > shard (MultiReader wraps 20 shards, so 20 tasks) but it was really
> > submitting
> > > a task for each segment within each shard (20 shards * ~10 segments =
> > ~200
> > > tasks) which is horrible. Since my index changes infrequently, I'm
> using
> > > forceMerge(1) before sending out updated indexes to the slave servers.
> > > Without any extra tuning (threads, # of shards, etc) I've gone from
> ~2900
> > > requests per minute to ~10k requests per minute.
> > >
> > > Thanks to Adrien and Mike for the clarification and Benson for bringing
> > up
> > > the question that led to my answer.
> > >
> > > I'm still pretty new to Lucene so I have a lot of poking around to do,
> > but I'm
> > > going to try to implement the "virtual segment" concept that Mike
> > > mentioned. It'll be really helpful for those of us who want parallelism
> > within
> > > queries and don't want to forceMerge.
> > >
> > >
> > > On Fri, Sep 27, 2013 at 9:55 AM, Desidero <desid...@gmail.com> wrote:
> > >
> > > > Erick,
> > > >
> > > > Thank you for responding.
> > > >
> > > > I ran tests using both compressed fields and uncompressed fields, and
> > > > it was significantly slower with uncompressed fields. I looked into
> > > > the lazy field loading per your suggestion, but we don't get any
> > > > values from the returned Documents until the result set has been
> > > appropriately reduced.
> > > > Since we only store one retrievable field and we always need to get
> > > > it, it doesn't save any time loading it lazily.
> > > >
> > > > I'll try running a test without loading any fields just to see how it
> > > > affects performance and let you know how that goes.
> > > >
> > > > Regards,
> > > > Matt
> > > >
> > > >
> > > > On Fri, Sep 27, 2013 at 8:01 AM, Erick Erickson
> > > <erickerick...@gmail.com>wrote:
> > > >
> > > >> Hmmm, since 4.1, fields have been stored compressed by default.
> > > >> I suppose it's possible that this is a result of
> > > >> compressing/uncompressing.
> > > >>
> > > >> What happens if
> > > >> 1> you enable lazy field loading
> > > >> 2> don't load any fields?
> > > >>
> > > >> FWIW,
> > > >> Erick
> > > >>
> > > >> On Thu, Sep 26, 2013 at 10:55 AM, Desidero <desid...@gmail.com>
> > > wrote:
> > > >> > A quick update:
> > > >> >
> > > >> > In order to confirm that none of the standard migration changes
> had
> > > >> > a negative effect on performance, I ported my Lucene 4.x version
> > > >> > back to Lucene 3.6.2 and kept the newer API rather than using the
> > > >> > custom ParallelMultiSearcher and other deprecated methods/classes.
> > > >> >
> > > >> > Performance in 3.6.2 is even faster than before (~2900
> requests/min
> > > >> with 4.x
> > > >> > vs ~6200 requests/min with 3.6.2), so none of my code changes
> > > >> > should be causing the difference. It seems to be something Lucene
> > > >> > is doing under
> > > >> the
> > > >> > covers.
> > > >> >
> > > >> > Again, if there's any other information if I can provide to help
> > > >> determine
> > > >> > what's going on, please let me know.
> > > >> >
> > > >> > Thanks,
> > > >> > Matt
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> -------------------------------------------------------------------
> > > >> > -- 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
> >
>

Reply via email to