prepareCommit does almost all the work, right up until writing a new segments_N file with all of its contents *except* the final checksum at the end of the file. Readers that try to open the index at this point will see an invalid checksum and will fallback to the previous segments_(N-1). Then, commit simply writes the checksum, closes & syncs the segments_N. Lucene doesn't rename files anymore (it's proven to be problematic on some platforms, eg Windows).

In the next (2.9) release of Lucene, you can pass IndexWriter an arbitrary IndexCommit to open, which effectively lets you rollback to any commit in the index. Ie, it's more powerful than rollback() which reverts changes in memory since the last commit or open.

Of course, you'll have to use a deletion policy that keeps more than one commit in the index. You can list all commits with IndexReader.listCommits.

It may also be useful to use prepareCommit(String) to save arbitrary String metadata into the commit, which you can then use later to tell them apart.

Mike

mark harwood wrote:


As suggested, the window for failure here is very small. The commit is effectively an atomic single file rename operation to make the new segments file visible. However, should there be a failure between 2 commits the new deletion policy logic should help you recover to prior commit points. See here for some discussion/Junit examples:

https://issues.apache.org/jira/browse/LUCENE-1449

Cheers
Mark



----- Original Message ----
From: "fang...@emc.com" <fang...@emc.com>
To: java-user@lucene.apache.org
Sent: Tuesday, 24 February, 2009 9:49:34
Subject: RE: IndexWriter 2-phase commit usage

The prepareCommit should do most real works, so the chance
index2.commit() failure should be slim.

I think it's very hard to compensate the changes already committed.
One solution is that you create separate indexes for each transaction
and merge them later.

Merging can fail, but the transaction still fulfill ACID.



-----Original Message-----
From: An Hong [mailto:an.h...@i365.com]
Sent: Tuesday, February 24, 2009 10:07 AM
To: lucene-user
Subject: IndexWriter 2-phase commit usage

I'm having trouble applying IndexWriter 2-phase commit to make a
transaction involving two different indexes.  The scenario,

1.  Open index1
2.  Open index2
3.  Make change1 to index1
4.  Make change2 to index2
5.  index1.prepareCommit() successfully flushes
6.  index2.prepareCommit() successfully flushes
7.  index1.commit() successfully (segmentN file written)
8.  index2.commit()
If step 8 fails, how can I roll back step 7 w/o implementing my own undo
logic?  I want to make change1 and change2 a single transaction.
Thanks,
An

---------------------------------------------------------------------
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