That what was I was hoping to get to eventually. I agree that flush can do much more (and be better renamed to prepare). If we can get something like that, it would be great. Just a note regarding two phase transactions, there is no guarantee (at least none that I am aware of) regarding the ordering of the first phase for XAResource, so having the lucene transaction as the last one is not easily implemented.
Michael McCandless-2 wrote: > > > That's a good point. > > Lucene is doing a 2-phase commit, under the hood in the commit() > method, but doesn't expose the two separate phases (prepare & commit) > through the API. > > Unfortunately, while the flush() call does some of the work of the > prepare phase, it's not doing enough. For example, it does not close > the "doc stores", which involve alot of IO if a compound file needs to > be built. It also does not fsync() all referenced files, which could > fail. Finally, it does not initiate writing the next segments_N file, > which can also fail if you run out of disk space right then. These > are really things that the "prepare" phase should be doing, because > they add risk of an IOException during commit(). > > Note that even lacking explicit exposure of the two separate phases of > the commit in Lucene's API, you can still involve Lucene in a > transaction with other resources (eg a database) that do expose the > two phases, by calling "IndexWriter.commit()" after all other > resources have successfully prepared, and then rolling back all other > resources on hitting an exception from Lucene, else committing them. > > So I don't think keeping flush(), and advertising it as the equivalent > of a prepare phase of a 2-phase commit protocol, is enough here -- it > would be "false advertising". > > But I agree it would be good for Lucene to explicitly expose the > "prepare" phase. Maybe we could add a "prepareCommit()" method, that > does the flush, and closes docs stores, and syncs, and initiates but > does not complete the writing of the next segments_N file. Having > called prepareCommit() you would not be allowed to call anything else > in IndexWriter until commit() or abort() is called. Also, if a > concurrent merges completes it would also be blocked from committing > the changes to the index until commit(). We should probably also > deprecate abort() and rename it to rollback(). I'll open an issue for > this... > > Mike > > Shay Banon wrote: >> >> Hi, >> >> I was just looking a bit at the trunk. First, let me say that the >> progress you guys make is amazing!. I would still like to ask a quick >> question regarding deprecation of flush in IndexWriter. I think >> that there >> are cases where flush is needed. For example, in trying to create a >> two >> phase (or as close as possible to one) commit. The flush can be >> used for the >> fist phase and the close/commit can be used for the second one. >> Does it make >> sense? >> >> Cheers, >> Shay >> -- >> View this message in context: http://www.nabble.com/Deprecation-of- >> flush-in-IndexWriter-tp16627610p16627610.html >> Sent from the Lucene - Java Developer mailing list archive at >> Nabble.com. >> >> >> --------------------------------------------------------------------- >> 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] > > > -- View this message in context: http://www.nabble.com/Deprecation-of-flush-in-IndexWriter-tp16627610p16648395.html Sent from the Lucene - Java Developer mailing list archive at Nabble.com. --------------------------------------------------------------------- To unsubscribe, e-mail: [EMAIL PROTECTED] For additional commands, e-mail: [EMAIL PROTECTED]
