-
Notifications
You must be signed in to change notification settings - Fork 2.7k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Alternative approach to LUCENE-8962 #1576
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -24,10 +24,16 @@ | |
import java.util.Map; | ||
import java.util.Map.Entry; | ||
import java.util.Set; | ||
import java.util.concurrent.CompletableFuture; | ||
import java.util.concurrent.ExecutionException; | ||
import java.util.concurrent.TimeUnit; | ||
import java.util.concurrent.TimeoutException; | ||
import java.util.concurrent.atomic.AtomicLong; | ||
import java.util.concurrent.locks.Condition; | ||
import java.util.concurrent.locks.ReentrantLock; | ||
import java.util.function.BiConsumer; | ||
import java.util.function.BooleanSupplier; | ||
import java.util.function.Consumer; | ||
import java.util.stream.Collectors; | ||
import java.util.stream.StreamSupport; | ||
|
||
|
@@ -196,6 +202,7 @@ final void setMergeThread(Thread owner) { | |
* | ||
* @lucene.experimental */ | ||
public static class OneMerge { | ||
private final CompletableFuture<Void> completable = new CompletableFuture<>(); | ||
SegmentCommitInfo info; // used by IndexWriter | ||
boolean registerDone; // used by IndexWriter | ||
long mergeGen; // used by IndexWriter | ||
|
@@ -222,13 +229,13 @@ public static class OneMerge { | |
volatile long mergeStartNS = -1; | ||
|
||
/** Total number of documents in segments to be merged, not accounting for deletions. */ | ||
public final int totalMaxDoc; | ||
final int totalMaxDoc; | ||
Throwable error; | ||
|
||
/** Sole constructor. | ||
* @param segments List of {@link SegmentCommitInfo}s | ||
* to be merged. */ | ||
public OneMerge(List<SegmentCommitInfo> segments) { | ||
/** Sole constructor. | ||
* @param segments List of {@link SegmentCommitInfo}s | ||
* to be merged. */ | ||
public OneMerge(List<SegmentCommitInfo> segments) { | ||
if (0 == segments.size()) { | ||
throw new RuntimeException("segments must include at least one segment"); | ||
} | ||
|
@@ -252,7 +259,24 @@ public void mergeInit() throws IOException { | |
} | ||
|
||
/** Called by {@link IndexWriter} after the merge is done and all readers have been closed. */ | ||
public void mergeFinished() throws IOException { | ||
public void mergeFinished(boolean committed) throws IOException { | ||
completable.complete(null); | ||
} | ||
|
||
boolean await(long timeout, TimeUnit timeUnit) { | ||
try { | ||
completable.get(timeout, timeUnit); | ||
return true; | ||
} catch (InterruptedException e) { | ||
Thread.interrupted(); | ||
return false; | ||
} catch (ExecutionException | TimeoutException e) { | ||
return false; | ||
} | ||
} | ||
|
||
boolean isDone() { | ||
return completable.isDone(); | ||
} | ||
|
||
/** Wrap the reader in order to add/remove information to the merged segment. */ | ||
|
@@ -399,8 +423,19 @@ public String segString(Directory dir) { | |
} | ||
return b.toString(); | ||
} | ||
|
||
boolean await(long timeout, TimeUnit unit) { | ||
for (OneMerge merge : merges) { | ||
if (merge.await(timeout, unit) == false) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This looks suspicious when there is more than one merge. Shouldn't the timeout decrease as time is used up by earlier merges? In practice, when is there more than one? I've been confused on this matter when I developed a custom MP/MS. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. in a real change that's correct. in a prototype as this is it's really just there to visualize the idea. I didn't do this on purpose to not discuss impl details. that's not the point of this it's really just a PR to make commenting simpler. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I fixed this in the followup PR #1585 |
||
return false; | ||
} | ||
} | ||
return true; | ||
} | ||
} | ||
|
||
|
||
|
||
/** Exception thrown if there are any problems while executing a merge. */ | ||
public static class MergeException extends RuntimeException { | ||
/** Create a {@code MergeException}. */ | ||
|
@@ -692,4 +727,27 @@ public interface MergeContext { | |
*/ | ||
Set<SegmentCommitInfo> getMergingSegments(); | ||
} | ||
|
||
/** | ||
* Identifies merges that we want to execute (synchronously) on commit. By default, do not synchronously merge on commit. | ||
* | ||
* Any merges returned here will make {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} block until | ||
* the merges complete or until {@link IndexWriterConfig#getMaxCommitMergeWaitSeconds()} have elapsed. This may be | ||
* used to merge small segments that have just been flushed as part of the commit, reducing the number of segments in | ||
* the commit. If a merge does not complete in the allotted time, it will continue to execute, but will not be reflected | ||
* in the commit. | ||
* | ||
* If a {@link OneMerge} in the returned {@link MergeSpecification} includes a segment already included in a registered | ||
* merge, then {@link IndexWriter#commit()} or {@link IndexWriter#prepareCommit()} will throw a {@link IllegalStateException}. | ||
* Use {@link MergeContext#getMergingSegments()} to determine which segments are currently registered to merge. | ||
* | ||
* @param mergeTrigger the event that triggered the merge (COMMIT or FULL_FLUSH). | ||
* @param segmentInfos the total set of segments in the index (while preparing the commit) | ||
* @param mergeContext the MergeContext to find the merges on, which should be used to determine which segments are | ||
* already in a registered merge (see {@link MergeContext#getMergingSegments()}). | ||
*/ | ||
public MergeSpecification findFullFlushMerges(MergeTrigger mergeTrigger, SegmentInfos segmentInfos, MergeContext mergeContext) throws IOException { | ||
return null; | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There is an inconsistency here that suggests something is wrong, or at least confusing enough to deserve a comment. For case COMMIT, we call a findFullFlushMerges. Shouldn't it be on case FULL_FLUSH to be consistent with the method we are calling? Or should findFullFlushMerges be called findCommitMerges?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Full flush happens for refresh and commit.
But we have only implemented "merge on commit" so far.
I would love to also add "merge on refresh", but until we do so, I think it's correct for
IndexWriter
to separate out theCOMMIT
case here like this.