lucene-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Michael McCandless <luc...@mikemccandless.com>
Subject Re: Atomic optimize() + commit()
Date Thu, 02 Apr 2009 15:02:16 GMT
With ConcurrentMergeScheduler, IndexWriter has gained alot of
concurrency, such that an optimize (or normal BG merge) could be
running at the same time as deletes/adds.  I think this is a good
thing and we should keep improving it (there are still places that
block, eg while a flush is running a merge cannot commit).

But, there are clearly cases where you want to explicitly prevent
concurrency operations (like your class that wraps
IndexWriter/Reader).  The current patch on LUCENE-1313 has something
similar, except in that case the atomic operation is "do adds, do
deletes, open new near-realtime reader".  Grant also proposed
generalizing IndexAccessor (in LUCENE-1516).

However: I think all such logic should live above
IndexWriter/IndexReader.  IndexWriter should try to be as concurrent
as possible, and if apps need further atomicity of certain groups of
operations, it should be done outside of Lucene's core.  Of course, if
IndexWriter doesn't expose enough APIs to enable such atomicity, we
should fix that.

I definitely agree we should fix commit's javadocs to include other
changes, like optimize() calls, addIndexes, etc. -- I'll do that.

Mike

On Thu, Apr 2, 2009 at 8:22 AM, Shai Erera <serera@gmail.com> wrote:
> Hi
>
> I've run into a problem in my code when I upgraded to 2.4. I am not sure if
> it is a real problem, but I thought I'd let you know anyway. The following
> is a background of how I ran into the issue, but I think the discussion does
> not necessarily involve my use of Lucene.
>
> I have a class which wraps all Lucene-related operations, i.e., addDocument,
> deleteDocument, search and optimize (those are the important ones for this
> email). It maintains an IndexWriter open, through which it does the
> add/delete/optimize operations and periodically opens an IndexReader for the
> search operations using the reopen() API.
>
> The application indexes operations (add, delete, update) by multiple
> threads, while there's a manager which after the last operation has been
> processed, calls commit, which does writer.commit(). I also check from time
> to time if the index needs to be optimized and optimizes if needed (the
> criteria for when to do it is irrelevant now). I also have a unit test which
> does several add/update/delete operations, calls optimize and checks the
> number of deleted documents. It expects to find 0, since optimize has been
> called and after I upgraded to 2.4 it failed.
>
> Now ... with the move to 2.4, I discovered that optimize() does not commit
> automatically and I have to call commit. It's a good place to say that when
> I was on 2.3 I used the default autoCommit=true and with the move to 2.4
> that default has changed, and being a good citizen, I also changed my code
> to call commit when I want and not use any deprecated ctors or rely on
> internal Lucene logic. I can only guess that that's why at the end of the
> test I still see numDeletedDocs != 0 (since optimize does not commit by
> default).
>
> So I went ahead and fixed my optimize() method to do: (1) writer.optimize()
> (2) writer.commit().
>
> But then I thought - is this fix correct? Is it the right approach? Suppose
> that at the sime time optimize was running, or just between (1) and (2)
> there was a context switch, and a thread added documents to the index. Upon
> calling commit(), the newly added documents are also committed, without the
> caller intending to do so. In my scenario this will probably not be too
> catastrophically, but I can imagine scenarios in which someone in addition
> to indexing updates a DB and has a virtual atomic commit, which commits the
> changes to the index as well as the DB, all the while locking any update
> operations. Suddenly that someone's code breaks.
>
> There are a couple of ways I can solve it, like for example synchronizing
> the optimize + commit on a lock which all indexing threads will also
> synchronize (allowing all of them to index concurrently, but if optimize is
> running all are blocked), but that will hold all my indexing threads. Or, I
> can just not call commit at the end, relying on the workers manager to
> commit at the next batch indexing work. However, during that time the
> readers will search on an unoptimized index, with deletes, while they can
> search on a freshly optimized index with no deletes (and less segments).
>
> The problem with those solutions is that they are not intuitive. To start
> with, the Lucene documentation itself is wrong - In IndexWriter.commit() it
> says: "Commits all pending updates (added & deleted documents)" - optimize
> is not mentioned (shouldn't this be fixed anyway?). Also, notice that the
> problem stems from the fact that the optimize operation may be called by
> another thread, not knowing there are update operations running. Lucene
> documents that you can call addDocument while optimize() is running, so
> there's no need to protect against that. Suddenly, we're requiring every
> search application developer to disregard the documentation and think to
> himself "do I want to allow optimize() to run concurrently with
> add/deletes?". I'm not saying that it's wrong, but if you're ok with it, we
> should document it.
>
> I wonder though if there isn't room to introduce an atomic optimize() +
> commit() in Lucene. The incentive is that optimize is not the same as
> add/delete. Add/delete are operations I may want to hide from my users,
> because they change the state of the index (i.e., how many searchable
> documents are there). Optimize just reorganizes the index, and is supposed
> to improve performance. When I call optimize, don't I want it to be
> committed? Will I ever want to hold that commit off (taking out edge cases)?
> I assume that 99.9% of the time that's what we expect from it.
>
> Now, just adding a call to commit() at the end of optimize() will not solve
> it, because that's the same as calling commit outside optimize(). We need
> the optimize's commit to only commit its changes. And if there are updates
> pending commit - not touch them.
>
> BTW, I've scanned through the documentation and haven't found any mention of
> such thing, however I may have still missed it. So if there is already a
> solution to that, or such an atomic optimize+commit, I apologize in advance
> for forcing you to read such a long email (for those of you who made it thus
> far) and appreciate if you give me a reference.
>
> Shai
>

---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Mime
View raw message