Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Previous Topic Next Topic
 
classic Classic list List threaded Threaded
8 messages Options
Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Michael McCandless-2
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

Review request for lucene.
By Mike McCandless.
Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (49dd333)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Re: Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Simon Willnauer-3
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (Diff revision 1)
126
    assert doc < maxDoc;
can you add a message to this assertion, if it trips we really wanna see what the values are?

lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (Diff revision 1)
210
      + values.bytes().length;
oh this is good!!

lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
168
    finishedDelGens.add(delGen);

it took me a while to figure out what you are doing here. I wonder if it makes sense to extract this in an inner class like:

static class FinishedSegments {
   private int completedDelGen;
   private final Set<Long> finishedDelGens = new HashSet<>();

   // do your thing here... 


}

I think that way you can also sync on this instance instead of using the synchronized on the BufferedUpdatesStream instance (nice sideeffect) and it's clear that the two members belong together. I also would appreciate if you could leave a comment why we need the hashset since we need to deal with potential holes in the finished segments history and that's why we need to wait until they are consecutive in order to be deleted.


lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
202
        if (packet != null && segGen < packet.delGen()) {
243
    if (waitFor.size() == 0) {
use `waitFor.isEmpty()` instead?

lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
276
  public void applyPacket(FrozenBufferedUpdates packet) throws IOException {

is it possible that an indexing thread is currently handling this? in that case we would do it twice? I'd love if you could add some in-line comment about that


lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
225
          SegmentState segState = segStates[infosIDX];
286
  private void innerApplyPacket(FrozenBufferedUpdates packet) throws IOException {

do we really need this inner method, can't we inline this into applyPacket


lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
342
        writer.deleter.incRef(delFiles);

this is pretty scary if you do that without a try / finally block. if we fail before we can decrement the refs we might leak the file references?

it's pretty unclear to me where it will be released and I wonder if we sould use some kind of datastructure where we can record all the files and make sure that if we exit this loop we release them? ie. have a single place where we release, like a list we add to?


lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
348
        if (segStates.length == 0) {
this loop is pretty scary while I get what you are doing. maybe it would be simpler if we can break out part of it into seperate methods? the size of the loop also makes it hard to reason about if we leak any resources

lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
295
      if (segStates != null) {
377
        synchronized (writer) {
this entire finally block should be a seperate method

lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (Diff revision 1)
720
      } catch (Throwable t) {
`catch(Throwable t)` really :) I know IW#tragicEvent will sort it out for us

lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (Diff revision 1)
442
    return documentsWriter.deleteQueue.ramBytesUsed() + bufferedUpdatesStream.ramBytesUsed();
436
    return documentsWriter.deleteQueue.ramBytesUsed();
oh that is great!

lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (Diff revision 1)
213
      notifyAll();
217
      //notifyAll();

is this a leftover?


lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 1)
246
  private long applyDocValuesUpdates(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
this really should have a dedicated untitest if possible

lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 1)
449
  private long applyQueryDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
I'd love to see dedicated unittests for these methods here if there aren't any. I mean it's private so maybe we should add some? It's pretty intense code here

lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 1)
590
            while (true) {
maybe:

```Java
int docID;
while ((docID = postingsEnum.nextDoc()) != NO_MORE_DOCS) {

}
```
?

lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 1)
593
        assert readerMap.containsKey(rld.info) == false;
an assertion message would be good here no?

lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 1)
2229
    if (maybeMergeLock.compareAndSet(false, true)) {
can you please use a real lock for this and not an atomicboolean...

you can still do `if (lock.tryLock())` there but it has clear semantics

lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (Diff revision 1)
95
95
    //System.out.println(Thread.currentThread().getName() + ": SCR init " + si.info.name);
unrelated debugging leftover, can you remove?

lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (Diff revision 1)
167
//      System.err.println("--- closing core readers");
167
      //System.out.println(Thread.currentThread().getName() + ": SCR close " + segment);

unrelated debugging leftover, can you remove?


lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (Diff revision 1)
165
      Directory dir;
is this a bug? should this be fixed seperately?

- Simon Willnauer


On June 16th, 2017, 1:20 p.m. UTC, Mike McCandless wrote:

Review request for lucene.
By Mike McCandless.

Updated June 16, 2017, 1:20 p.m.

Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (49dd333)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Re: Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Michael McCandless-2
In reply to this post by Michael McCandless-2
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

Review request for lucene.
By Mike McCandless.

Updated June 18, 2017, 1:22 p.m.

Changes

Another patch folding in Simon's feedback.

Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs (updated)

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (49dd333)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Show Changes

Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Re: Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Simon Willnauer-3
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 2)
332
      totalDelCount += delCount;

can we already assign this here? it's going to be counted more than once when we retry no? I am not sure I understand why we add this up here maybe it's a leftover?


lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 2)
373
      String message = String.format(Locale.ROOT,

if we really want to log the sum of the del docs we should also log how many iterations we took?


lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 2)
813
    assert this.delGen == -1;

a message would be helpful here?


lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 2)
2236
    if (maybeMergeLock.tryLock()) {

so lets say we index heavily and we have a heavy flushing party going on but we are kind of skipping most of the maybeMerge calls since we only wanna do this once and don't block etc. we might miss a merge when the last thread comes in here but skips since it can't acquire the lock. I think to be absolutely correct here we also need to retry if there is a thread that couldn't lock the maybeMergeLock otherwise we might miss to kick off a merge? I mean in practice this might not be a massive issue but testing will fail at some point here.

We can piggyback on the AtomicBoolean maybeMerge and set it to true if we can't acquire the lock here. if the value is still true after we called mergeScheduler.merge(this, trigger, newMergesFound); we retry but we should release the lock first. something like this:

do {
  maybeMerge.set(true); // the thread checking will retry
  if (maybeMergeLock.tryLock()) {
    if (maybeMerge.compareAndSet(true, false)) { // only try if we have to... some other thread might have been faster?
      maybeMerge.set(false); // we are checking now if there is another trigger coming it will be true again
      try {
        boolean newMergesFound = updatePendingMerges(mergePolicy, trigger, maxNumSegments);
        mergeScheduler.merge(this, trigger, newMergesFound);
      } finally {
        maybeMergeLock.unlock();
      }
    }
  }
} while (maybeMerge.get()); // retry until there is no pending merge check


lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 2)
3110
          doMaybeMerge[0] = true;
3289
          maybeMerge.set(true);
man this `boolean[]` is/was scary :)

lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 2)
3627
    bufferedUpdatesStream.waitApplyAll();

should we assert that we don't hold the IW lock here.. afaik it could be an issue and deadlock?


lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (Diff revision 2)
95
  public final static boolean DEFAULT_READER_POOLING = false;
95
  public final static boolean DEFAULT_READER_POOLING = true;

it would be good for me and the future readers to have an inline comment here when we changed this to true and why. Changes like this are not obvious and we really need to prepare for others to catch up with this. We already have a crazy bus-factor on IW.


lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (Diff revision 2)
428
  public IndexWriterConfig setMaxBufferedDeleteTerms(int maxBufferedDeleteTerms) {
this is great, it was super complex to account for all these settings.

lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (Diff revision 2)
261
    assert Thread.holdsLock(writer);

is this assertino on the IW lock not necessary anymore? I recall you saying you didn't parallelize the file writing yet!?


lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (Diff revision 2)
838
      assert delCount > reader.numDeletedDocs();

a message would be great here?


- Simon Willnauer


On June 18th, 2017, 1:22 p.m. UTC, Mike McCandless wrote:

Review request for lucene.
By Mike McCandless.

Updated June 18, 2017, 1:22 p.m.

Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (49dd333)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Re: Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Michael McCandless-2
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

On June 19th, 2017, 10:16 a.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 2)
332
      totalDelCount += delCount;

can we already assign this here? it's going to be counted more than once when we retry no? I am not sure I understand why we add this up here maybe it's a leftover?

Good question! It smells funny, but it's actually safe; I'll add comments clarifying why.


On June 19th, 2017, 10:16 a.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 2)
373
      String message = String.format(Locale.ROOT,

if we really want to log the sum of the del docs we should also log how many iterations we took?

I do log that (add to message, right below this) if it's more than one.


On June 19th, 2017, 10:16 a.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 2)
2236
    if (maybeMergeLock.tryLock()) {

so lets say we index heavily and we have a heavy flushing party going on but we are kind of skipping most of the maybeMerge calls since we only wanna do this once and don't block etc. we might miss a merge when the last thread comes in here but skips since it can't acquire the lock. I think to be absolutely correct here we also need to retry if there is a thread that couldn't lock the maybeMergeLock otherwise we might miss to kick off a merge? I mean in practice this might not be a massive issue but testing will fail at some point here.

We can piggyback on the AtomicBoolean maybeMerge and set it to true if we can't acquire the lock here. if the value is still true after we called mergeScheduler.merge(this, trigger, newMergesFound); we retry but we should release the lock first. something like this:

do {
  maybeMerge.set(true); // the thread checking will retry
  if (maybeMergeLock.tryLock()) {
    if (maybeMerge.compareAndSet(true, false)) { // only try if we have to... some other thread might have been faster?
      maybeMerge.set(false); // we are checking now if there is another trigger coming it will be true again
      try {
        boolean newMergesFound = updatePendingMerges(mergePolicy, trigger, maxNumSegments);
        mergeScheduler.merge(this, trigger, newMergesFound);
      } finally {
        maybeMergeLock.unlock();
      }
    }
  }
} while (maybeMerge.get()); // retry until there is no pending merge check

Hmm, I tried this, but it resulting in infinite loops for some tests; then I tried to improve it, but still infinite loops.

Then, I simply reverted it (back to trunk) and it works again! And the reason I had made this change was in fact invalid: it was due to a separate bug (now fixed) that was causing way too much segment flushing, so I think it's best to just leave it as it was :)


On June 19th, 2017, 10:16 a.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 2)
3110
          doMaybeMerge[0] = true;
3289
          maybeMerge.set(true);
man this `boolean[]` is/was scary :)

YES!!


On June 19th, 2017, 10:16 a.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (Diff revision 2)
3627
    bufferedUpdatesStream.waitApplyAll();

should we assert that we don't hold the IW lock here.. afaik it could be an issue and deadlock?

Good catch, I'll add.


On June 19th, 2017, 10:16 a.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (Diff revision 2)
95
  public final static boolean DEFAULT_READER_POOLING = false;
95
  public final static boolean DEFAULT_READER_POOLING = true;

it would be good for me and the future readers to have an inline comment here when we changed this to true and why. Changes like this are not obvious and we really need to prepare for others to catch up with this. We already have a crazy bus-factor on IW.

Amen! I'll add a comment.


On June 19th, 2017, 10:16 a.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (Diff revision 2)
261
    assert Thread.holdsLock(writer);

is this assertino on the IW lock not necessary anymore? I recall you saying you didn't parallelize the file writing yet!?

Good catch, I'll put it back. And yes, unfortunately, no concurrent delete/update file writes yet.


- Mike


On June 18th, 2017, 1:22 p.m. UTC, Mike McCandless wrote:

Review request for lucene.
By Mike McCandless.

Updated June 18, 2017, 1:22 p.m.

Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (49dd333)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Re: Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Michael McCandless-2
In reply to this post by Michael McCandless-2
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

Review request for lucene.
By Mike McCandless.

Updated June 21, 2017, 10:04 a.m.

Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs (updated)

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (1503de8)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Show Changes

Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Re: Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Michael McCandless-2
In reply to this post by Simon Willnauer-3
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
276
  public void applyPacket(FrozenBufferedUpdates packet) throws IOException {

is it possible that an indexing thread is currently handling this? in that case we would do it twice? I'd love if you could add some in-line comment about that

I sync on the packet and checked the applied CountDownLatch to see if it was completed already, so it should never happen twice.

I'll move this whole method into FrozenBufferedUpdates to make that sync clearer.


On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
342
        writer.deleter.incRef(delFiles);

this is pretty scary if you do that without a try / finally block. if we fail before we can decrement the refs we might leak the file references?

it's pretty unclear to me where it will be released and I wonder if we sould use some kind of datastructure where we can record all the files and make sure that if we exit this loop we release them? ie. have a single place where we release, like a list we add to?

OK, I've changed the incRef to be the last thing we do in the sync(IW) block, and the matched decRef to the first thing I do in the finally block.


On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
348
        if (segStates.length == 0) {
this loop is pretty scary while I get what you are doing. maybe it would be simpler if we can break out part of it into seperate methods? the size of the loop also makes it hard to reason about if we leak any resources

OK, I factored out two chunks of code to separate methods.


On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (Diff revision 1)
295
      if (segStates != null) {
377
        synchronized (writer) {
this entire finally block should be a seperate method

OK I did that.


On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (Diff revision 1)
720
      } catch (Throwable t) {
`catch(Throwable t)` really :) I know IW#tragicEvent will sort it out for us

Yeah :) I think an exception here is necessarily tragic because the internal state of the packet has changed and we cannot easily recover/retry applying it...


On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (Diff revision 1)
213
      notifyAll();
217
      //notifyAll();

is this a leftover?

Yeah, I intend to switch to notify. I think the notifyAll defensively is too much: only one thread state freed up.


On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (Diff revision 1)
449
  private long applyQueryDeletes(BufferedUpdatesStream.SegmentState[] segStates) throws IOException {
I'd love to see dedicated unittests for these methods here if there aren't any. I mean it's private so maybe we should add some? It's pretty intense code here

I think TestNumeric/BinaryDocValuesUpdates and TestIndexWriterDelete/ByQuery do a pretty good job testing here?


On June 16th, 2017, 8:31 p.m. UTC, Simon Willnauer wrote:

lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (Diff revision 1)
165
      Directory dir;
is this a bug? should this be fixed seperately?

Not I bug: I just refactored the ternary operator up above to a real if.


- Mike


On June 21st, 2017, 10:04 a.m. UTC, Mike McCandless wrote:

Review request for lucene.
By Mike McCandless.

Updated June 21, 2017, 10:04 a.m.

Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (1503de8)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Reply | Threaded
Open this post in threaded view
|  
Report Content as Inappropriate

Re: Review Request 60154: LUCENE-7868: Concurrent deletes and doc values updates

Simon Willnauer-3
In reply to this post by Michael McCandless-2
This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/60154/

Ship it!

Ship It!

- Simon Willnauer


On June 21st, 2017, 10:04 a.m. UTC, Mike McCandless wrote:

Review request for lucene.
By Mike McCandless.

Updated June 21, 2017, 10:04 a.m.

Repository: lucene-solr

Description

Today Lucene uses a single thread to resolve buffered delete/update terms to actual docIDs, but this is a costly process. This change uses multiple threads (the incoming indexing threads) to resolve terms concurrently.

Jira issue: https://issues.apache.org/jira/browse/LUCENE-7868

Testing

I ran performance tests on my internal corpus; details described here: https://issues.apache.org/jira/browse/LUCENE-7868?focusedCommentId=16050729&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16050729

I also uncovered a pre-existing bug if you use DV updates with index sorting and update recently indexed documents; I'll open a separate issue to fix that on 6.x.

Diffs

  • lucene/core/src/java/org/apache/lucene/index/BinaryDocValuesFieldUpdates.java (f8cece9)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdates.java (1c3494f)
  • lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (9955626)
  • lucene/core/src/java/org/apache/lucene/index/CoalescedUpdates.java (bf92ac1)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesFieldUpdates.java (528d4bf)
  • lucene/core/src/java/org/apache/lucene/index/DocValuesUpdate.java (1c85f33)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriter.java (2807517)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java (db0e571)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java (a5b4b7c)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushQueue.java (2c62487)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (c929ba2)
  • lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java (cc72342)
  • lucene/core/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java (a85c98b)
  • lucene/core/src/java/org/apache/lucene/index/FlushPolicy.java (e70959f)
  • lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (1ca2830)
  • lucene/core/src/java/org/apache/lucene/index/FrozenBufferedUpdates.java (4f482ad)
  • lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java (f7f196d)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriter.java (14fbbae)
  • lucene/core/src/java/org/apache/lucene/index/IndexWriterConfig.java (0fdbc3e)
  • lucene/core/src/java/org/apache/lucene/index/LiveIndexWriterConfig.java (d9e1bc7)
  • lucene/core/src/java/org/apache/lucene/index/MergedPrefixCodedTermsIterator.java (cd14eec)
  • lucene/core/src/java/org/apache/lucene/index/NumericDocValuesFieldUpdates.java (4dd3cd0)
  • lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java (df1653b)
  • lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java (d4dd4a4)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCommitInfo.java (b1084a6)
  • lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java (ce2d448)
  • lucene/core/src/java/org/apache/lucene/index/SegmentInfo.java (1c02441)
  • lucene/core/src/java/org/apache/lucene/index/SegmentReader.java (c8235d5)
  • lucene/core/src/java/org/apache/lucene/index/SerialMergeScheduler.java (5a8f98b)
  • lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (668f1ec)
  • lucene/core/src/java/org/apache/lucene/util/packed/AbstractPagedMutable.java (c5fac1e)
  • lucene/core/src/test/org/apache/lucene/index/TestBinaryDocValuesUpdates.java (ed2b66f)
  • lucene/core/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java (c60f54d)
  • lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java (aa2901c)
  • lucene/core/src/test/org/apache/lucene/index/TestForceMergeForever.java (0379395)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriter.java (6897f06)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterConfig.java (2014c16)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (25817d9)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java (c0907a5)
  • lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (584e03c)
  • lucene/core/src/test/org/apache/lucene/index/TestNRTReaderWithThreads.java (871715f)
  • lucene/core/src/test/org/apache/lucene/index/TestNumericDocValuesUpdates.java (94da587)
  • lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java (112a108)
  • lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java (89d4ad1)
  • lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java (a1b2a5c)
  • lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (1503de8)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java (334f784)
  • lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java (d83b915)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (8cb6665)
  • lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java (959466a)
  • lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (0243a56)

View Diff

Loading...