I pushed a fix.  Sorry!

On Mon, Aug 18, 2014 at 10:39 AM, Uwe Schindler <[email protected]> wrote:
> Hi,
>
> you broke build in 4.x: There is still a reference to finishMerges().
>
> Uwe
>
> -----
> Uwe Schindler
> H.-H.-Meier-Allee 63, D-28213 Bremen
> http://www.thetaphi.de
> eMail: [email protected]
>
>
>> -----Original Message-----
>> From: [email protected] [mailto:[email protected]]
>> Sent: Monday, August 18, 2014 7:20 PM
>> To: [email protected]
>> Subject: svn commit: r1618668 - in /lucene/dev/branches/branch_4x: ./
>> lucene/ lucene/core/
>> lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
>> lucene/core/src/test/org/apache/lucene/index/TestIndexWriterMergePolic
>> y.java
>>
>> Author: rjernst
>> Date: Mon Aug 18 17:20:27 2014
>> New Revision: 1618668
>>
>> URL: http://svn.apache.org/r1618668
>> Log:
>> Backport fix in waitForMerges to allow merge scheduler to run one last time
>>
>> Modified:
>>     lucene/dev/branches/branch_4x/   (props changed)
>>     lucene/dev/branches/branch_4x/lucene/   (props changed)
>>     lucene/dev/branches/branch_4x/lucene/core/   (props changed)
>>
>> lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/i
>> ndex/IndexWriter.java
>>
>> lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/i
>> ndex/TestIndexWriterMergePolicy.java
>>
>> Modified:
>> lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/i
>> ndex/IndexWriter.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/cor
>> e/src/java/org/apache/lucene/index/IndexWriter.java?rev=1618668&r1=161
>> 8667&r2=1618668&view=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/i
>> ndex/IndexWriter.java (original)
>> +++
>> lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene
>> +++ /index/IndexWriter.java Mon Aug 18 17:20:27 2014
>> @@ -907,7 +907,11 @@ public class IndexWriter implements Clos
>>            infoStream.message("IW", "now flush at close");
>>          }
>>          flush(true, true);
>> -        finishMerges(waitForMerges);
>> +        if (waitForMerges) {
>> +          waitForMerges();
>> +        } else {
>> +          abortMerges();
>> +        }
>>          commitInternal(config.getMergePolicy());
>>          rollbackInternal(); // ie close, since we just committed
>>          success = true;
>> @@ -2145,7 +2149,7 @@ public class IndexWriter implements Clos
>>
>>      try {
>>        synchronized(this) {
>> -        finishMerges(false);
>> +        abortMerges();
>>          stopMerges = true;
>>        }
>>
>> @@ -2282,7 +2286,7 @@ public class IndexWriter implements Clos
>>          synchronized (this) {
>>            try {
>>              // Abort any running merges
>> -            finishMerges(false);
>> +            abortMerges();
>>              // Remove all segments
>>              segmentInfos.clear();
>>              // Ask deleter to locate unreferenced files & remove them:
>> @@ -2316,56 +2320,48 @@ public class IndexWriter implements Clos
>>      }
>>    }
>>
>> -  private synchronized void finishMerges(boolean waitForMerges) {
>> -    if (!waitForMerges) {
>> +  /** Aborts running merges.  Be careful when using this
>> +   *  method: when you abort a long-running merge, you lose
>> +   *  a lot of work that must later be redone. */  public synchronized
>> + void abortMerges() {
>> +    stopMerges = true;
>>
>> -      stopMerges = true;
>> -
>> -      // Abort all pending & running merges:
>> -      for (final MergePolicy.OneMerge merge : pendingMerges) {
>> -        if (infoStream.isEnabled("IW")) {
>> -          infoStream.message("IW", "now abort pending merge " +
>> segString(merge.segments));
>> -        }
>> -        merge.abort();
>> -        mergeFinish(merge);
>> +    // Abort all pending & running merges:
>> +    for (final MergePolicy.OneMerge merge : pendingMerges) {
>> +      if (infoStream.isEnabled("IW")) {
>> +        infoStream.message("IW", "now abort pending merge " +
>> + segString(merge.segments));
>>        }
>> -      pendingMerges.clear();
>> +      merge.abort();
>> +      mergeFinish(merge);
>> +    }
>> +    pendingMerges.clear();
>>
>> -      for (final MergePolicy.OneMerge merge : runningMerges) {
>> -        if (infoStream.isEnabled("IW")) {
>> -          infoStream.message("IW", "now abort running merge " +
>> segString(merge.segments));
>> -        }
>> -        merge.abort();
>> +    for (final MergePolicy.OneMerge merge : runningMerges) {
>> +      if (infoStream.isEnabled("IW")) {
>> +        infoStream.message("IW", "now abort running merge " +
>> + segString(merge.segments));
>>        }
>> +      merge.abort();
>> +    }
>>
>> -      // These merges periodically check whether they have
>> -      // been aborted, and stop if so.  We wait here to make
>> -      // sure they all stop.  It should not take very long
>> -      // because the merge threads periodically check if
>> -      // they are aborted.
>> -      while(runningMerges.size() > 0) {
>> -        if (infoStream.isEnabled("IW")) {
>> -          infoStream.message("IW", "now wait for " + runningMerges.size() + 
>> "
>> running merge/s to abort");
>> -        }
>> -        doWait();
>> +    // These merges periodically check whether they have
>> +    // been aborted, and stop if so.  We wait here to make
>> +    // sure they all stop.  It should not take very long
>> +    // because the merge threads periodically check if
>> +    // they are aborted.
>> +    while(runningMerges.size() > 0) {
>> +      if (infoStream.isEnabled("IW")) {
>> +        infoStream.message("IW", "now wait for " + runningMerges.size()
>> + + " running merge/s to abort");
>>        }
>> +      doWait();
>> +    }
>>
>> -      stopMerges = false;
>> -      notifyAll();
>> -
>> -      assert 0 == mergingSegments.size();
>> +    stopMerges = false;
>> +    notifyAll();
>>
>> -      if (infoStream.isEnabled("IW")) {
>> -        infoStream.message("IW", "all running merges have aborted");
>> -      }
>> +    assert 0 == mergingSegments.size();
>>
>> -    } else {
>> -      // waitForMerges() will ensure any running addIndexes finishes.
>> -      // It's fine if a new one attempts to start because from our
>> -      // caller above the call will see that we are in the
>> -      // process of closing, and will throw an
>> -      // AlreadyClosedException.
>> -      waitForMerges();
>> +    if (infoStream.isEnabled("IW")) {
>> +      infoStream.message("IW", "all running merges have aborted");
>>      }
>>    }
>>
>> @@ -2375,20 +2371,30 @@ public class IndexWriter implements Clos
>>     * <p>It is guaranteed that any merges started prior to calling this 
>> method
>>     *    will have completed once this method completes.</p>
>>     */
>> -  public synchronized void waitForMerges() {
>> -    ensureOpen(false);
>> -    if (infoStream.isEnabled("IW")) {
>> -      infoStream.message("IW", "waitForMerges");
>> -    }
>> -    while(pendingMerges.size() > 0 || runningMerges.size() > 0) {
>> -      doWait();
>> -    }
>> +  public void waitForMerges() throws IOException {
>>
>> -    // sanity check
>> -    assert 0 == mergingSegments.size();
>> +    // Give merge scheduler last chance to run, in case
>> +    // any pending merges are waiting. We can't hold IW's lock
>> +    // when going into merge because it can lead to deadlock.
>> +    mergeScheduler.merge(this, MergeTrigger.CLOSING, false);
>>
>> -    if (infoStream.isEnabled("IW")) {
>> -      infoStream.message("IW", "waitForMerges done");
>> +    synchronized (this) {
>> +      ensureOpen(false);
>> +      if (infoStream.isEnabled("IW")) {
>> +        infoStream.message("IW", "waitForMerges");
>> +      }
>> +
>> +
>> +      while (pendingMerges.size() > 0 || runningMerges.size() > 0) {
>> +        doWait();
>> +      }
>> +
>> +      // sanity check
>> +      assert 0 == mergingSegments.size();
>> +
>> +      if (infoStream.isEnabled("IW")) {
>> +        infoStream.message("IW", "waitForMerges done");
>> +      }
>>      }
>>    }
>>
>> @@ -4049,7 +4055,7 @@ public class IndexWriter implements Clos
>>     *  the synchronized lock on IndexWriter instance. */
>>    final synchronized void mergeFinish(MergePolicy.OneMerge merge) {
>>
>> -    // forceMerge, addIndexes or finishMerges may be waiting
>> +    // forceMerge, addIndexes or waitForMerges may be waiting
>>      // on merges to finish.
>>      notifyAll();
>>
>>
>> Modified:
>> lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/i
>> ndex/TestIndexWriterMergePolicy.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/cor
>> e/src/test/org/apache/lucene/index/TestIndexWriterMergePolicy.java?rev=
>> 1618668&r1=1618667&r2=1618668&view=diff
>> ==========================================================
>> ====================
>> ---
>> lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/i
>> ndex/TestIndexWriterMergePolicy.java (original)
>> +++
>> lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene
>> +++ /index/TestIndexWriterMergePolicy.java Mon Aug 18 17:20:27 2014
>> @@ -233,7 +233,7 @@ public class TestIndexWriterMergePolicy
>>      writer.addDocument(doc);
>>    }
>>
>> -  private void checkInvariants(IndexWriter writer) {
>> +  private void checkInvariants(IndexWriter writer) throws IOException {
>>      writer.waitForMerges();
>>      int maxBufferedDocs = writer.getConfig().getMaxBufferedDocs();
>>      int mergeFactor = ((LogMergePolicy)
>> writer.getConfig().getMergePolicy()).getMergeFactor();
>
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: [email protected]
> For additional commands, e-mail: [email protected]
>

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to