You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by Doron Cohen <DO...@il.ibm.com> on 2007/01/16 01:01:21 UTC

allowing applications to control docids change? (e.g. setKeepDeletes(boolean)?)

Note: discussion started originally in
http://www.nabble.com/adding-%22explicit-commits%22-to-Lucene--t3011270.html


robert engels <re...@ix.netcom.com> wrote on 15/01/2007 13:23:14:

> I think that you will find a much larger performance decrease in
> doing things this way - if the external resource is a db, or any
> networked accessed resource.

That's possible - let's see - I had two scenarios in mind:

Scenario A:
(1) documents are maintained in a database;
(2) each document is not small - it may have lots of text;
(3) some parsing may be required before tokenizing;
(4) there are many documents;
(5) there is a specific document property <P> which is used during search,
for, say - filtering.
(6) <P> may change often, and re-indexing documents just because of that is
too expensive.

Scenario B:
(1) to (5) as above.
(6) <P> is computable from (say) a database, but is unsteady and cannot be
indexed - this is the case that came out in
http://www.gossamer-threads.com/lists/lucene/java-user/44122
(7) <P> cannot be used for post filtering, because search results without
that filtering might return too many false results.
(8) there may be (frequent) document updates.

If we accept the assumption that maintaining the changing value in the
index is either impossible or inefficient, the application would strive to
maintain these values outside of Lucene. Then:

Scenario A:
- Two 'arrays' can be maintained - <P>, and <B> (==isDeleted)
- At search, <P> and <B> are read and used to construct a filter, possibly
as an IndexReader (propriately) implementation, and used with e.g.
ParallelReader.
- At calling optimize, The two arrays are used for creating the 'next' two
arrays.

Scenario B:
- After thinking more about this, I agree with you for this scenario -
updating the mapping after optimize is not trivial. (I first thought that
this should be easy since we know which docs were updated, but I now see
that this would be very expensive.)

So, this leaves one scenario that would benefit from this. If Lucene had
in-place-update that would not be needed, but this does not seems likely in
the near future.

Okay, this is quite a stressed example, however a real one.

If people think that this may be indeed useful, I can go on and see what it
means to implement something like this.

>
> When even just a single document is changed in the Lucene index you
> could have MILLIONS of changes to internal doc ids (if say an early
> document was deleted).
>
> Seems far better to store the external id in the Lucene index.
>
> You will find that performance penalty of looking up the Lucene
> document by the external id (vs. the internal doc #), to be far less
> than the performance penalty of updating every document in the
> external index when the Lucene index is merged.
>
> The only case I can see this would be of any benefit is if the Lucene
> index RARELY if EVER changes - anything else, and you will have big
> problems.
>
> Now, if the Lucene is changed to support point in time searching
> (basically never delete any index files), you might be able to do
> what you this. Just create a Directory only creating the segments up
> to that time.
>
> Sounds VERY messy to me.
>
> On Jan 15, 2007, at 3:12 PM, Doron Cohen wrote:
>
> > Also related is the request made several times in the list to be
> > able to
> > control when docids are changing, for applications that need to
> > maintain
> > some mapping between external IDs to Lucene docs but for some
> > performance
> > reasons cannot afford to only count on storing external (DB) IDs in
> > Lucene's field. For instance, recent discussion "Making document
> > numbers
> > persistent" in java-user.
> >
> > So, an application controlled commit would allow an application not to
> > "experience" document numbering changes - no docid changes would
> > affect the
> > application until a commit is issued. So the application would be
> > able to
> > call optimize and then issue a commit, thereby exposing docid changes.
> >
> > One disadvantage of controlling ids changes like this is that
> > search would
> > have to stale long behind index updates, unless optimize is called.
> >
> > Therefore, - that's another issue of course - I am wondering if
> > there might
> > be interest in allowing applications to control whether deleted
> > docs are
> > allowed to be removed/squeezed-out or not.
> >
> > Michael McCandless <lu...@mikemccandless.com> wrote on 14/01/2007
> > 13:36:34:
> >
> >> Team,
> >>
> >> I've been struggling to find a clean solution for LUCENE-710, when I
> >> thought of a simple addition to Lucene ("explicit commits") that
> >> would
> >> I think resolve LUCENE-710 and would fix a few other outstanding
> >> issues when readers are using a "live" index (being updated by a
> >> writer).
> >>
> >> The basic idea is to add an explicit "commit" operation to Lucene.
> >>
> >> This is the same nice feature Solr has, but just a different
> >> implementation (in Lucene core, in a single index, instead).  The
> >> commit makes a "point in time" snapshot (term borrowed from Solr!)
> >> available for searching.
> >>
> >> The implementation is surprisingly simple (see below) and completely
> >> backwards compatible.
> >>
> >> I'd like to get some feedback on the idea/implementation.
> >>
> >>
> >> Details...: right now, Lucene writes a new segments_N file at various
> >> times: when a writer (or reader that's writing deletes/norms)
> >> needs to
> >> flush its pending changes to disk; when a writer merges segments;
> >> when
> >> a writer is closed; multiple times during optimize/addIndexes; etc.
> >>
> >> These times are not controllable / predictable to the developer using
> >> Lucene.
> >>
> >> A new reader always opens the last segments_N written, and, when a
> >> reader uses isCurrent() to check whether it should re-open (the
> >> suggested way), that method always returns false (meaning you should
> >> re-open) if there are any new segments_N files.
> >>
> >> So it's somewhat uncontrollable to the developer what state the index
> >> is in when you [re-]open a reader.
> >>
> >> People work around this today by adding logic above Lucene so that
> >> the
> >> writer separately communicates to readers when is a good time to
> >> refresh.  But with "explicit commits", readers could instead look
> >> directly at the index and pick the right segments_N to refresh to.
> >>
> >> I'm proposing that we separate the writing of a new segments_N file
> >> into those writes that are done automatically by Lucene (I'll call
> >> these "checkpoints") from meaningful (to the application) commits
> >> that
> >> are done explicitly by the developer at known times (I'll call this
> >> "committing a snapshot").  I would add a new boolean mode to
> >> IndexWriter called "autoCommit", and a new public method "commit
> >> ()" to
> >> IndexWriter and IndexReader (we'd have to rename the current
> >> protected
> >> commit() in IndexReader)
> >>
> >> When autoCommit is true, this means every write of a segments_N file
> >> will be "commit a snapshot", meaning readers will then use it for
> >> searching.  This will be the default and this is exactly how Lucene
> >> behaves today.  So this change is completely backwards compatible.
> >>
> >> When autoCommit is false, then when Lucene chooses to save a
> >> segments_N file it's just a "checkpoint": a reader would not open or
> >> re-open to the checkpoint.  This means the developer must then call
> >> IndexWriter.commit() or IndexReader.commit() in order to "commit a
> >> snapshot" at the right time, thereby telling readers that this
> >> segments_N file is a valid one to switch to for searching.
> >>
> >>
> >> The implementation is very simple (I have an initial coarse prototype
> >> working with all but the last bullet):
> >>
> >>    * If a segments_N file is just a checkpoint, it's named
> >>      "segmentsx_N" (note the added 'x'); if it's a snapshot, it's
> >> named
> >>      "segments_N".  No other changes to the index format.
> >>
> >>    * A reader by default opens the latest snapshot but can optionally
> >>      open a specific N (segments_N) snapshot.
> >>
> >>    * A writer by default starts from the most recent "checkpoint" but
> >>      may also take a specific checkpoint or snapshot point N
> >>      (segments_N) to start from (to allow rollback).
> >>
> >>    * Change IndexReader.isCurrent() to see if there are any newer
> >>      snapshots but disregard newer checkpoints.
> >>
> >>    * When a writer is in autoCommit=false mode, it always writes
> >> to the
> >>      next segmentsx_N; else it writes to segments_N.
> >>
> >>    * The commit() method would just write to the next segments_N file
> >>      and return the N it had written (in case application needs to
> >>      re-use it later).
> >>
> >>    * IndexFileDeleter would need to have a slightly smarter policy
> >> when
> >>      autoCommit=false, ie, "don't delete anything referenced by
> >> either
> >>      the past N snapshots or if the snapshot was obsoleted less
> >> than X
> >>      minutes ago".
> >>
> >>
> >> I think there are some compelling things this could solve:
> >>
> >>    * The "delete then add" problem (really a special but very common
> >>      case of general transactions):
> >>
> >>      Right now when you want to update a bunch of documents in a
> >> Lucene
> >>      index, it's best to open a reader, do a "batch delete", close
> >> the
> >>      reader, open a writer, do a "batch add", close the writer.  This
> >>      is the suggested way.
> >>
> >>      The open risk here is that a reader could refresh at any time
> >>      during these operations, and find that a bunch of documents have
> >>      been deleted but not yet added again.
> >>
> >>      Whereas, with autoCommit false you could do this entire
> >> operation
> >>      (batch delete then batch add), and then call the final commit
> >> () in
> >>      the end, and readers would know not to re-open the index until
> >>      that final commit() succeeded.
> >>
> >>    * The "using too much disk space during optimize" problem:
> >>
> >>      This came up on the user's list recently: if you aggressively
> >>      refresh readers while optimize() is running, you can tie up much
> >>      more disk space than you'd expect, because your readers are
> >>      holding open all the [possibly very large] intermediate
> >> segments.
> >>
> >>      Whereas, if autoCommit is false, then developer calls optimize()
> >>      and then calls commit(), the readers would know not to re-open
> >>      until optimize was complete.
> >>
> >>    * More general transactions:
> >>
> >>      It has come up a fair number of times how to make Lucene
> >>      transactional, either by itself ("do the following complex
> >> series
> >>      of index operations but if there is any failure, rollback to the
> >>      start, and don't expose result to searcher until all operations
> >>      are done") or as part of a larger transaction eg involving a
> >>      relational database.
> >>
> >>      EG, if you want to add a big set of documents to Lucene, but not
> >>      make them searchable until they are all added, or until a
> >> specific
> >>      time (eg Monday @ 9 AM), you can't do that easily today but it
> >>      would be simple with explicit commits.
> >>
> >>      I believe this change would make transactions work correctly
> >> with
> >>      Lucene.
> >>
> >>    * LUCENE-710 ("implement point in time searching without
> >> relying on
> >>      filesystem semantics"), also known as "getting Lucene to work
> >>      correctly over NFS".
> >>
> >>      I think this issue is nearly solved when autoCommit=false, as
> >> long
> >>      as we can adopt a shared policy on "when readers refresh" to
> >> match
> >>      the new deletion policy (described above).  Basically, as
> >> long as
> >>      the deleter and readers are playing by the same "refresh rules"
> >>      and the writer gives the readers enough time to switch/warm,
> >> then
> >>      the deleter should never delete something in use by a reader.
> >>
> >>
> >>
> >> There are also some neat future things made possible:
> >>
> >>    * The "support deleteDocuments in IndexWriter" (LUCENE-565)
> >> feature
> >>      could have a more efficient implementation (just like Solr) when
> >>      autoCommit is false, because deletes don't need to be flushed
> >>      until commit() is called.  Whereas, now, they must be
> >> aggressively
> >>      flushed on each checkpoint.
> >>
> >>    * More generally, because "checkpoints" do not need to be
> >> usable by
> >>      a reader/searcher, other neat optimizations might be possible.
> >>
> >>      EG maybe the merge policy could be improved if it knows that
> >>      certain segments are "just checkpoints" and are not involved in
> >>      searching.
> >>
> >>    * I could simplify the approach for my recent addIndexes changes
> >>      (LUCENE-702) to use this, instead of it's current approach
> >> (wish I
> >>      had thought of this sooner: ugh!.).
> >>
> >>    * A single index could hold many snapshots, and, we could enable a
> >>      reader to explicitly open against an older snapshot.  EG
> >> maybe you
> >>      take weekly and a monthly snapshot because you sometimes want to
> >>      go back and "run a search on last week's catalog".
> >>
> >> Feedback?
> >>
> >> Mike
> >>


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


Re: allowing applications to control docids change? (e.g. setKeepDeletes(boolean)?)

Posted by Doron Cohen <DO...@il.ibm.com>.
robert engels <re...@ix.netcom.com> wrote on 15/01/2007 16:37:35:

> I did a cursory review of the discussion.
>
> The problem I see is that in the checkpoint tx files you need a
> 'delete file' for every segment where a deletion SHOULD occur when it
> is commited, but if you have multiple open transactions being
> created, as soon as one is applied (committed), the deletions being
> tracked in the other tx are no longer valid. This would imply that
> only a single tx can be active, and if that is the case, there are
> easier methods.

My example (scenario A) can in fact be a non database application (the
other characteristics remain).

As a database application, to my understanding the (newly suggested)
transaction support in Lucene is single tx. I can't see how multiple tx can
be done within Lucene (and I don't think it should be done). Even if it was
possible, I think indexing would become very inefficient. I think the
motivation for adding (some) tx support is different, and tx support would
be minimal, definitely not multiple tx.

>
> Simple example:
>
> Consider the index with documents A (doc 0), and B (doc 1) stored in
> a single segment (S1).
>
> User 1 open a tx to modify I, deletes A and inserts C. tx needs to
> track that 0 is deleted, C is added.
> User 2 open a tx to modify I, deletes B and inserts D. tx needs to
> track that 1 is deleted, D is added.
>
> tx1 commits, and optimizes, creates a new segment S2, containing B
> (doc 0), C (doc 1).
>
> Now tx2 commits:
>
> It cannot apply its changes to S2, since if doc 1 is deleted,
> document C will be removed from the index.
>
> It can't figure out what doc numbers in S2 correspond to doc numbers
> in S1 either (since there is no unique key).
>
> How do you propose to solve this? Am I missing something here?
>
>
> On Jan 15, 2007, at 6:01 PM, Doron Cohen wrote:
>
> >
> > Note: discussion started originally in
> > http://www.nabble.com/adding-%22explicit-commits%22-to-Lucene--
> > t3011270.html
> >
> >
> > robert engels <re...@ix.netcom.com> wrote on 15/01/2007 13:23:14:
> >
> >> I think that you will find a much larger performance decrease in
> >> doing things this way - if the external resource is a db, or any
> >> networked accessed resource.
> >
> > That's possible - let's see - I had two scenarios in mind:
> >
> > Scenario A:
> > (1) documents are maintained in a database;
> > (2) each document is not small - it may have lots of text;
> > (3) some parsing may be required before tokenizing;
> > (4) there are many documents;
> > (5) there is a specific document property <P> which is used during
> > search,
> > for, say - filtering.
> > (6) <P> may change often, and re-indexing documents just because of
> > that is
> > too expensive.
> >
> > Scenario B:
> > (1) to (5) as above.
> > (6) <P> is computable from (say) a database, but is unsteady and
> > cannot be
> > indexed - this is the case that came out in
> > http://www.gossamer-threads.com/lists/lucene/java-user/44122
> > (7) <P> cannot be used for post filtering, because search results
> > without
> > that filtering might return too many false results.
> > (8) there may be (frequent) document updates.
> >
> > If we accept the assumption that maintaining the changing value in the
> > index is either impossible or inefficient, the application would
> > strive to
> > maintain these values outside of Lucene. Then:
> >
> > Scenario A:
> > - Two 'arrays' can be maintained - <P>, and <B> (==isDeleted)
> > - At search, <P> and <B> are read and used to construct a filter,
> > possibly
> > as an IndexReader (propriately) implementation, and used with e.g.
> > ParallelReader.
> > - At calling optimize, The two arrays are used for creating the
> > 'next' two
> > arrays.
> >
> > Scenario B:
> > - After thinking more about this, I agree with you for this scenario -
> > updating the mapping after optimize is not trivial. (I first
> > thought that
> > this should be easy since we know which docs were updated, but I
> > now see
> > that this would be very expensive.)
> >
> > So, this leaves one scenario that would benefit from this. If
> > Lucene had
> > in-place-update that would not be needed, but this does not seems
> > likely in
> > the near future.
> >
> > Okay, this is quite a stressed example, however a real one.
> >
> > If people think that this may be indeed useful, I can go on and see
> > what it
> > means to implement something like this.
> >
> >>
> >> When even just a single document is changed in the Lucene index you
> >> could have MILLIONS of changes to internal doc ids (if say an early
> >> document was deleted).
> >>
> >> Seems far better to store the external id in the Lucene index.
> >>
> >> You will find that performance penalty of looking up the Lucene
> >> document by the external id (vs. the internal doc #), to be far less
> >> than the performance penalty of updating every document in the
> >> external index when the Lucene index is merged.
> >>
> >> The only case I can see this would be of any benefit is if the Lucene
> >> index RARELY if EVER changes - anything else, and you will have big
> >> problems.
> >>
> >> Now, if the Lucene is changed to support point in time searching
> >> (basically never delete any index files), you might be able to do
> >> what you this. Just create a Directory only creating the segments up
> >> to that time.
> >>
> >> Sounds VERY messy to me.
> >>


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


Re: allowing applications to control docids change? (e.g. setKeepDeletes(boolean)?)

Posted by robert engels <re...@ix.netcom.com>.
I did a cursory review of the discussion.

The problem I see is that in the checkpoint tx files you need a  
'delete file' for every segment where a deletion SHOULD occur when it  
is commited, but if you have multiple open transactions being  
created, as soon as one is applied (committed), the deletions being  
tracked in the other tx are no longer valid. This would imply that  
only a single tx can be active, and if that is the case, there are  
easier methods.

Simple example:

Consider the index with documents A (doc 0), and B (doc 1) stored in  
a single segment (S1).

User 1 open a tx to modify I, deletes A and inserts C. tx needs to  
track that 0 is deleted, C is added.
User 2 open a tx to modify I, deletes B and inserts D. tx needs to  
track that 1 is deleted, D is added.

tx1 commits, and optimizes, creates a new segment S2, containing B  
(doc 0), C (doc 1).

Now tx2 commits:

It cannot apply its changes to S2, since if doc 1 is deleted,  
document C will be removed from the index.

It can't figure out what doc numbers in S2 correspond to doc numbers  
in S1 either (since there is no unique key).

How do you propose to solve this? Am I missing something here?


On Jan 15, 2007, at 6:01 PM, Doron Cohen wrote:

>
> Note: discussion started originally in
> http://www.nabble.com/adding-%22explicit-commits%22-to-Lucene-- 
> t3011270.html
>
>
> robert engels <re...@ix.netcom.com> wrote on 15/01/2007 13:23:14:
>
>> I think that you will find a much larger performance decrease in
>> doing things this way - if the external resource is a db, or any
>> networked accessed resource.
>
> That's possible - let's see - I had two scenarios in mind:
>
> Scenario A:
> (1) documents are maintained in a database;
> (2) each document is not small - it may have lots of text;
> (3) some parsing may be required before tokenizing;
> (4) there are many documents;
> (5) there is a specific document property <P> which is used during  
> search,
> for, say - filtering.
> (6) <P> may change often, and re-indexing documents just because of  
> that is
> too expensive.
>
> Scenario B:
> (1) to (5) as above.
> (6) <P> is computable from (say) a database, but is unsteady and  
> cannot be
> indexed - this is the case that came out in
> http://www.gossamer-threads.com/lists/lucene/java-user/44122
> (7) <P> cannot be used for post filtering, because search results  
> without
> that filtering might return too many false results.
> (8) there may be (frequent) document updates.
>
> If we accept the assumption that maintaining the changing value in the
> index is either impossible or inefficient, the application would  
> strive to
> maintain these values outside of Lucene. Then:
>
> Scenario A:
> - Two 'arrays' can be maintained - <P>, and <B> (==isDeleted)
> - At search, <P> and <B> are read and used to construct a filter,  
> possibly
> as an IndexReader (propriately) implementation, and used with e.g.
> ParallelReader.
> - At calling optimize, The two arrays are used for creating the  
> 'next' two
> arrays.
>
> Scenario B:
> - After thinking more about this, I agree with you for this scenario -
> updating the mapping after optimize is not trivial. (I first  
> thought that
> this should be easy since we know which docs were updated, but I  
> now see
> that this would be very expensive.)
>
> So, this leaves one scenario that would benefit from this. If  
> Lucene had
> in-place-update that would not be needed, but this does not seems  
> likely in
> the near future.
>
> Okay, this is quite a stressed example, however a real one.
>
> If people think that this may be indeed useful, I can go on and see  
> what it
> means to implement something like this.
>
>>
>> When even just a single document is changed in the Lucene index you
>> could have MILLIONS of changes to internal doc ids (if say an early
>> document was deleted).
>>
>> Seems far better to store the external id in the Lucene index.
>>
>> You will find that performance penalty of looking up the Lucene
>> document by the external id (vs. the internal doc #), to be far less
>> than the performance penalty of updating every document in the
>> external index when the Lucene index is merged.
>>
>> The only case I can see this would be of any benefit is if the Lucene
>> index RARELY if EVER changes - anything else, and you will have big
>> problems.
>>
>> Now, if the Lucene is changed to support point in time searching
>> (basically never delete any index files), you might be able to do
>> what you this. Just create a Directory only creating the segments up
>> to that time.
>>
>> Sounds VERY messy to me.
>>
>> On Jan 15, 2007, at 3:12 PM, Doron Cohen wrote:
>>
>>> Also related is the request made several times in the list to be
>>> able to
>>> control when docids are changing, for applications that need to
>>> maintain
>>> some mapping between external IDs to Lucene docs but for some
>>> performance
>>> reasons cannot afford to only count on storing external (DB) IDs in
>>> Lucene's field. For instance, recent discussion "Making document
>>> numbers
>>> persistent" in java-user.
>>>
>>> So, an application controlled commit would allow an application  
>>> not to
>>> "experience" document numbering changes - no docid changes would
>>> affect the
>>> application until a commit is issued. So the application would be
>>> able to
>>> call optimize and then issue a commit, thereby exposing docid  
>>> changes.
>>>
>>> One disadvantage of controlling ids changes like this is that
>>> search would
>>> have to stale long behind index updates, unless optimize is called.
>>>
>>> Therefore, - that's another issue of course - I am wondering if
>>> there might
>>> be interest in allowing applications to control whether deleted
>>> docs are
>>> allowed to be removed/squeezed-out or not.
>>>
>>> Michael McCandless <lu...@mikemccandless.com> wrote on 14/01/2007
>>> 13:36:34:
>>>
>>>> Team,
>>>>
>>>> I've been struggling to find a clean solution for LUCENE-710,  
>>>> when I
>>>> thought of a simple addition to Lucene ("explicit commits") that
>>>> would
>>>> I think resolve LUCENE-710 and would fix a few other outstanding
>>>> issues when readers are using a "live" index (being updated by a
>>>> writer).
>>>>
>>>> The basic idea is to add an explicit "commit" operation to Lucene.
>>>>
>>>> This is the same nice feature Solr has, but just a different
>>>> implementation (in Lucene core, in a single index, instead).  The
>>>> commit makes a "point in time" snapshot (term borrowed from Solr!)
>>>> available for searching.
>>>>
>>>> The implementation is surprisingly simple (see below) and  
>>>> completely
>>>> backwards compatible.
>>>>
>>>> I'd like to get some feedback on the idea/implementation.
>>>>
>>>>
>>>> Details...: right now, Lucene writes a new segments_N file at  
>>>> various
>>>> times: when a writer (or reader that's writing deletes/norms)
>>>> needs to
>>>> flush its pending changes to disk; when a writer merges segments;
>>>> when
>>>> a writer is closed; multiple times during optimize/addIndexes; etc.
>>>>
>>>> These times are not controllable / predictable to the developer  
>>>> using
>>>> Lucene.
>>>>
>>>> A new reader always opens the last segments_N written, and, when a
>>>> reader uses isCurrent() to check whether it should re-open (the
>>>> suggested way), that method always returns false (meaning you  
>>>> should
>>>> re-open) if there are any new segments_N files.
>>>>
>>>> So it's somewhat uncontrollable to the developer what state the  
>>>> index
>>>> is in when you [re-]open a reader.
>>>>
>>>> People work around this today by adding logic above Lucene so that
>>>> the
>>>> writer separately communicates to readers when is a good time to
>>>> refresh.  But with "explicit commits", readers could instead look
>>>> directly at the index and pick the right segments_N to refresh to.
>>>>
>>>> I'm proposing that we separate the writing of a new segments_N file
>>>> into those writes that are done automatically by Lucene (I'll call
>>>> these "checkpoints") from meaningful (to the application) commits
>>>> that
>>>> are done explicitly by the developer at known times (I'll call this
>>>> "committing a snapshot").  I would add a new boolean mode to
>>>> IndexWriter called "autoCommit", and a new public method "commit
>>>> ()" to
>>>> IndexWriter and IndexReader (we'd have to rename the current
>>>> protected
>>>> commit() in IndexReader)
>>>>
>>>> When autoCommit is true, this means every write of a segments_N  
>>>> file
>>>> will be "commit a snapshot", meaning readers will then use it for
>>>> searching.  This will be the default and this is exactly how Lucene
>>>> behaves today.  So this change is completely backwards compatible.
>>>>
>>>> When autoCommit is false, then when Lucene chooses to save a
>>>> segments_N file it's just a "checkpoint": a reader would not  
>>>> open or
>>>> re-open to the checkpoint.  This means the developer must then call
>>>> IndexWriter.commit() or IndexReader.commit() in order to "commit a
>>>> snapshot" at the right time, thereby telling readers that this
>>>> segments_N file is a valid one to switch to for searching.
>>>>
>>>>
>>>> The implementation is very simple (I have an initial coarse  
>>>> prototype
>>>> working with all but the last bullet):
>>>>
>>>>    * If a segments_N file is just a checkpoint, it's named
>>>>      "segmentsx_N" (note the added 'x'); if it's a snapshot, it's
>>>> named
>>>>      "segments_N".  No other changes to the index format.
>>>>
>>>>    * A reader by default opens the latest snapshot but can  
>>>> optionally
>>>>      open a specific N (segments_N) snapshot.
>>>>
>>>>    * A writer by default starts from the most recent  
>>>> "checkpoint" but
>>>>      may also take a specific checkpoint or snapshot point N
>>>>      (segments_N) to start from (to allow rollback).
>>>>
>>>>    * Change IndexReader.isCurrent() to see if there are any newer
>>>>      snapshots but disregard newer checkpoints.
>>>>
>>>>    * When a writer is in autoCommit=false mode, it always writes
>>>> to the
>>>>      next segmentsx_N; else it writes to segments_N.
>>>>
>>>>    * The commit() method would just write to the next segments_N  
>>>> file
>>>>      and return the N it had written (in case application needs to
>>>>      re-use it later).
>>>>
>>>>    * IndexFileDeleter would need to have a slightly smarter policy
>>>> when
>>>>      autoCommit=false, ie, "don't delete anything referenced by
>>>> either
>>>>      the past N snapshots or if the snapshot was obsoleted less
>>>> than X
>>>>      minutes ago".
>>>>
>>>>
>>>> I think there are some compelling things this could solve:
>>>>
>>>>    * The "delete then add" problem (really a special but very  
>>>> common
>>>>      case of general transactions):
>>>>
>>>>      Right now when you want to update a bunch of documents in a
>>>> Lucene
>>>>      index, it's best to open a reader, do a "batch delete", close
>>>> the
>>>>      reader, open a writer, do a "batch add", close the writer.   
>>>> This
>>>>      is the suggested way.
>>>>
>>>>      The open risk here is that a reader could refresh at any time
>>>>      during these operations, and find that a bunch of documents  
>>>> have
>>>>      been deleted but not yet added again.
>>>>
>>>>      Whereas, with autoCommit false you could do this entire
>>>> operation
>>>>      (batch delete then batch add), and then call the final commit
>>>> () in
>>>>      the end, and readers would know not to re-open the index until
>>>>      that final commit() succeeded.
>>>>
>>>>    * The "using too much disk space during optimize" problem:
>>>>
>>>>      This came up on the user's list recently: if you aggressively
>>>>      refresh readers while optimize() is running, you can tie up  
>>>> much
>>>>      more disk space than you'd expect, because your readers are
>>>>      holding open all the [possibly very large] intermediate
>>>> segments.
>>>>
>>>>      Whereas, if autoCommit is false, then developer calls  
>>>> optimize()
>>>>      and then calls commit(), the readers would know not to re-open
>>>>      until optimize was complete.
>>>>
>>>>    * More general transactions:
>>>>
>>>>      It has come up a fair number of times how to make Lucene
>>>>      transactional, either by itself ("do the following complex
>>>> series
>>>>      of index operations but if there is any failure, rollback  
>>>> to the
>>>>      start, and don't expose result to searcher until all  
>>>> operations
>>>>      are done") or as part of a larger transaction eg involving a
>>>>      relational database.
>>>>
>>>>      EG, if you want to add a big set of documents to Lucene,  
>>>> but not
>>>>      make them searchable until they are all added, or until a
>>>> specific
>>>>      time (eg Monday @ 9 AM), you can't do that easily today but it
>>>>      would be simple with explicit commits.
>>>>
>>>>      I believe this change would make transactions work correctly
>>>> with
>>>>      Lucene.
>>>>
>>>>    * LUCENE-710 ("implement point in time searching without
>>>> relying on
>>>>      filesystem semantics"), also known as "getting Lucene to work
>>>>      correctly over NFS".
>>>>
>>>>      I think this issue is nearly solved when autoCommit=false, as
>>>> long
>>>>      as we can adopt a shared policy on "when readers refresh" to
>>>> match
>>>>      the new deletion policy (described above).  Basically, as
>>>> long as
>>>>      the deleter and readers are playing by the same "refresh  
>>>> rules"
>>>>      and the writer gives the readers enough time to switch/warm,
>>>> then
>>>>      the deleter should never delete something in use by a reader.
>>>>
>>>>
>>>>
>>>> There are also some neat future things made possible:
>>>>
>>>>    * The "support deleteDocuments in IndexWriter" (LUCENE-565)
>>>> feature
>>>>      could have a more efficient implementation (just like Solr)  
>>>> when
>>>>      autoCommit is false, because deletes don't need to be flushed
>>>>      until commit() is called.  Whereas, now, they must be
>>>> aggressively
>>>>      flushed on each checkpoint.
>>>>
>>>>    * More generally, because "checkpoints" do not need to be
>>>> usable by
>>>>      a reader/searcher, other neat optimizations might be possible.
>>>>
>>>>      EG maybe the merge policy could be improved if it knows that
>>>>      certain segments are "just checkpoints" and are not  
>>>> involved in
>>>>      searching.
>>>>
>>>>    * I could simplify the approach for my recent addIndexes changes
>>>>      (LUCENE-702) to use this, instead of it's current approach
>>>> (wish I
>>>>      had thought of this sooner: ugh!.).
>>>>
>>>>    * A single index could hold many snapshots, and, we could  
>>>> enable a
>>>>      reader to explicitly open against an older snapshot.  EG
>>>> maybe you
>>>>      take weekly and a monthly snapshot because you sometimes  
>>>> want to
>>>>      go back and "run a search on last week's catalog".
>>>>
>>>> Feedback?
>>>>
>>>> Mike
>>>>
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-dev-help@lucene.apache.org
>


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