Skip to content
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

LUCENE-9414 EagerCheapMergePolicy #1222

Closed
wants to merge 3 commits into from

Conversation

dsmiley
Copy link
Contributor

@dsmiley dsmiley commented Jan 29, 2020

https://issues.apache.org/jira/browse/LUCENE-8962
(contributed by Salesforce)

See early comment:
// Extends TieredMergePolicy for convenience but we could adjust to delegate or modify TieredMergePolicy

(contributed by Salesforce)
@@ -0,0 +1,202 @@
/*
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we move this to sandbox instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do you think? It'd be nice for users to get an implementation of findFullFlushMerges without requiring sandbox.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this a rather dangerous merge policy, at least as I currently understand it, so I think baking for a while in sandbox so we can confirm for more general users this is indeed safe.

* okay if there are multiple merge threads (thus less likely to reach such thresholds) or if it's
* deemed more important to do cheap merges at the expense of slowing index throughput further.
*/
public EagerCheapMergePolicy setCheapLimitConcurrentMergeSegments(int cheapLimitConcurrentMergeSegments) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this is somewhat dangerous .... when a large merge kicks off, and runs for minutes maybe, it means we would suppress these "cheap merges on refresh/commit" while it is running?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes but why is that dangerous? The intent of this was to protect from excessive merging / i/o.

Copy link
Member

@mikemccand mikemccand Jun 22, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Dangerous because it will lead to merge starvation for some users (maybe not SalesForce...).

Those tiny/fast merges really need to run at all times. If you stop them because a huge merge is running it can lead to an explosion of small segments in the index, GC problems, running out of file descriptors, etc. This is risky because you might not notice this when testing in dev, but then in production after indexing for a long time, it may strike.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, I was confused here -- since this merge policy is only picking (cheap) merges during commit, it is not as dangerous as I thought. I had thought it was doing this in general (for natural and forced merges too).

return infosSortedAsc.subList(0, numSegmentsToMerge);
}

private static long size(SegmentCommitInfo infoPerCommit) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Rename to sizeInBytes?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Intentionally this is named consistently with MergePolicy.size(). Otherwise I'd prefer the unit as well.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's not propagate the bad naming? Just because an existing method is poorly named doesn't mean we should name new methods poorly too :) Maybe open an issue to deprecate/rename MergePolicy.size?

Also, why not just use MergePolicy.size instead of adding a new dangerous method?

InfoStream infoStream = mergeContext.getInfoStream();
boolean verbose = infoStream.isEnabled(INFO_STREAM_CATEGORY);

if (mergeTrigger == MergeTrigger.FULL_FLUSH) { // i.e. commit
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also refresh?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't see MergeTrigger.REFRESH; maybe you meant something else? Any way this is now obsolete given that I updated this PR to use findFullFlushMerges

@dsmiley dsmiley changed the title LUCENE-8962 EagerCheapMergePolicy LUCENE-9414 EagerCheapMergePolicy Jun 22, 2020
Copy link
Member

@mikemccand mikemccand left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this merge policy is confusingly mixing two concepts. Concept #1 is merging on commit (the powerful newly added Lucene feature). Concept #2 is only picking merges if too many other merges are not already running.

I think this is not a good way to expose merge-on-commit by default ... it would be better to simply pick the smallish merges regardless of what other merges are taking place. The newly flushed segments are likely typically still hot (cached in RAM in OS's IO cache) and could be merged without any additional IO.

*/
public EagerCheapMergePolicy setCheapMaxSizeRatio(double cheapMaxSizeRatio) {
if (cheapMaxSizeRatio <= 0.0) {
throw new IllegalArgumentException("must be > 0");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Include the actual (invalid) value that was passed in here, in the exception message?

boolean verbose = infoStream.isEnabled(INFO_STREAM_CATEGORY);

if (mergeContext.getMergingSegments().size() <= cheapLimitConcurrentMergeSegments) {
OneMerge cheapMerge = findCheapMerge(segmentInfos, mergeContext);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, why only one merge? If enough concurrent threads are indexing, there could be many small segments written during commit.

return infosSortedAsc.subList(0, numSegmentsToMerge);
}

private static long size(SegmentCommitInfo infoPerCommit) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's not propagate the bad naming? Just because an existing method is poorly named doesn't mean we should name new methods poorly too :) Maybe open an issue to deprecate/rename MergePolicy.size?

Also, why not just use MergePolicy.size instead of adding a new dangerous method?

}

private static long size(SegmentCommitInfo infoPerCommit) {
//note: We could discount for deleted docs (as TieredMergePolicy does) but lets keep this simple/cheap.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But it is important for a merge policy to favor merging segments with many deletions? Or, are you thinking that on commit the newly flushed segments will not have many deletions typically so why bother? I think it would be simpler / less risky to just use the existing MergePolicy.size?

import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.StringHelper;

public class EagerCheapMergePolicyTest extends LuceneTestCase {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You could also insert it into LuceneTestCase.newIndexWriterConfig maybe?

* search over. This policy only overrides {@link #findFullFlushMerges(MergeTrigger, SegmentInfos, MergeContext)}; the
* rest are delegated to another policy.
*/
public class EagerCheapMergePolicy extends FilterMergePolicy {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we rename this to MergeOnCommitMergePolicy perhaps?

Copy link
Contributor Author

@dsmiley dsmiley Jun 22, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's a nice vote of confidence I suppose because that's the generic name and it'd mean attempts from others to add a MP that merges on commit likewise would get the most obvious name. For example "Highlighter" is forever people's go-to highlighter despite there being a better option. If you like it so much, I do :-), maybe TieredMergePolicy might have this algorithm and a setMaxCommitMergeWaitSeconds of 0 would mean it's never called any way and is thus "safe" to those who don't want it. Just a thought and perhaps deserves JIRA level commentary instead of hidden in a GitHub PR code review.

* okay if there are multiple merge threads (thus less likely to reach such thresholds) or if it's
* deemed more important to do cheap merges at the expense of slowing index throughput further.
*/
public EagerCheapMergePolicy setCheapLimitConcurrentMergeSegments(int cheapLimitConcurrentMergeSegments) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Actually, I was confused here -- since this merge policy is only picking (cheap) merges during commit, it is not as dangerous as I thought. I had thought it was doing this in general (for natural and forced merges too).

@dsmiley
Copy link
Contributor Author

dsmiley commented Jun 22, 2020

The setCheapLimitConcurrentMergeSegments limit was simply a precautionary addition given that I knew these cheap merges would up the write amplification and so I wanted to counter-act it in ways. Basically, I wanted this MP to not be "dangerous".

@dsmiley dsmiley closed this Aug 2, 2023
@dsmiley dsmiley deleted the lucene8962_mergePolicy_eager branch August 2, 2023 22:12
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants