-
Notifications
You must be signed in to change notification settings - Fork 25k
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
Clean up commits when global checkpoint advanced #28140
Clean up commits when global checkpoint advanced #28140
Conversation
Today we keep multiple index commits based on the global checkpoint, but only clean up old index commits when we have a new index commit. We however can release unneeded index commits earlier once the global checkpoint has advanced enough. This commit revisits the index deletion policy whenever a new global checkpoint value is persisted.
# Conflicts: # server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java # server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java # server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java
# Conflicts: # server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java
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.
Thanks @dnhatn . I left some feedback.
/** | ||
* Checks if the deletion policy can release some index commits with the latest global checkpoint. | ||
*/ | ||
synchronized boolean hasUnreferencedCommits() throws IOException { |
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.
a couple of comments here:
- does this needs to be synchronized? can't we use volatile for safeCommit?
- can we also pre-empt the long parsing etc with a check whether safeCommit != lastCommit? without it this will keep on returning true when the safe commit is safe and is the only commit?
- Do won't we want to check for the moment when the lastCommit becomes safe? I'm a bit confused by the current implementation. Under normal circumstances the global checkpoint is > max seq no for the safe commit?
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.
@bleskes Very good catch. This implementation is incorrect. We should compare the global checkpoint to the max_seqno of a commit after the safe commit.
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.
@dnhatn just to be clear, I don't think we should be heroic and start doing the "optimal" thing of checking the commit after the safe commit (which will mean starting to store more info). I think we can rely on just checking the last commit. It will become safe very quickly.
/** | ||
* This method should be called after the translog has been synced. | ||
*/ | ||
public void onTranslogSynced() throws IOException { |
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.
I think we should do this differently, rather than sync the translog directly, we can make the translog sync method package private and have all syncs go through the engine. wdyt?
@@ -2318,6 +2318,7 @@ protected void write(List<Tuple<Translog.Location, Consumer<Exception>>> candida | |||
try { | |||
final Engine engine = getEngine(); | |||
engine.getTranslog().ensureSynced(candidates.stream().map(Tuple::v1)); | |||
engine.onTranslogSynced(); |
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.
this only need to happened if ensuredSync returned true.
@bleskes I've updated the comparison and made translog-sync methods go through the engine. However, Translog and Engine are in different packages, I had to keep translog-sync methods public. I made sure that Engine is the only consumer. Please give it another look when you have time. Thank you! |
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.
Looks great. Left some very minor comments
revisitIndexDeletionPolicy(); | ||
} | ||
|
||
private void revisitIndexDeletionPolicy() throws IOException { |
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.
revisitIndexDeletionPolicyOnTranslogSync?
IndexCommit safeCommit = randomFrom(commitList); | ||
globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO))); | ||
indexPolicy.onCommit(commitList); | ||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), lastMaxSeqNo)); // Advanced not enough |
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.
why is this not enough? we use >= in our check?
if (safeCommit == commitList.get(commitList.size() - 1)) { | ||
assertThat(indexPolicy.hasUnreferencedCommits(), equalTo(false)); // Keeping a single commit | ||
} else { | ||
assertThat(indexPolicy.hasUnreferencedCommits(), equalTo(true)); |
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.
can we run another on commit and check that we just have one commit left and that the indexPolicy. hasUnreferencedCommits now returns true?
for (int docId = 0; docId < numDocs; docId++) { | ||
index(engine, docId); | ||
if (rarely()) { | ||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); |
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.
don't we have to use engine.getLocalCheckpointTracker().getCheckpoint()-1
to make sure the rest goes well and the GCP doesn't go backwards?
This does not bring special value but makes test harder to read.
@bleskes Your comments are addressed. Please give it another go. Thank you. |
please test this. |
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.
LGTM. Thanks Nhat
/** | ||
* Checks if the deletion policy can release some index commits with the latest global checkpoint. | ||
*/ | ||
boolean hasUnreferencedCommits() throws IOException { |
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.
just an idea for a follow up, shall we extend this method to look at the fact that a snapshotted commit was released (via a special flag we set when a snapshot count reaches 0)? we need to figure where to call it, but I think might be worth exploring.
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.
@bleskes, I am thinking to always revisit the index deletion policy (without checking the unreferenced condition) when releasing a snapshotted commit in an Engine (InternalEngine#acquireIndexCommit). We don't acquire index commits too frequently and revisiting the policy should not be expensive. WDYT?
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.
@dnhatn I think I wasn't clear. The idea was to add a "pendingSnapshots" flag that is set when a snapshot reference goes to 0 and is cleared onCommit.
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.
Thanks @bleskes. I got the idea.
Thanks @bleskes for your helpful reviews. |
* es/master: (38 commits) Build: Add pom generation to meta plugins (#28321) Add 6.3 version constant to master Minor improvements to translog docs (#28237) [Docs] Remove typo in painless-getting-started.asciidoc Build: Fix meta plugin usage in integ test clusters (#28307) Painless: Add spi jar that will be published for extending whitelists (#28302) mistyping in one of the highlighting examples comment -> content (#28139) Documents applicability of term query to range type (#28166) Build: Omit dependency licenses check for elasticsearch deps (#28304) Clean up commits when global checkpoint advanced (#28140) Implement socket and server ChannelContexts (#28275) Plugins: Fix meta plugins to install bundled plugins with their real name (#28285) Build: Fix meta plugin integ test installation (#28286) Modify Abstract transport tests to use impls (#28270) Fork Groovy compiler onto compile Java home [Docs] Update tophits-aggregation.asciidoc (#28273) Docs: match between snippet to its description (#28296) [TEST] fix RequestTests#testSearch in case search source is not set REST high-level client: remove index suffix from indices client method names (#28263) Fix simple_query_string on invalid input (#28219) ...
Today we keep multiple index commits based on the current global checkpoint, but only clean up unneeded index commits when we have a new index commit. However, we can release the old index commits earlier once the global checkpoint has advanced enough. This commit makes an engine revisit the index deletion policy whenever a new global checkpoint value is persisted and advanced enough. Relates #10708
* 6.x: Trim down usages of `ShardOperationFailedException` interface (#28312) Clean up commits when global checkpoint advanced (#28140) Do not return all indices if a specific alias is requested via get aliases api. CountedBitSet doesn't need to extend BitSet. (#28239) Calculate sum in Kahan summation algorithm in aggregations (#27807) (#27848)
A follow-up of elastic#28140 We currently revisit the index deletion policy whenever the global checkpoint has advanced enough. However, we won't be able to clean up the old commit points if they are being snapshotted. Here we prefer a simple solution over an optimal solution as we should revisit if only the last commit is being snapshotted.
We currently revisit the index deletion policy whenever the global checkpoint has advanced enough. We should also revisit the deletion policy after releasing the last snapshot of a snapshotting commit. With this change, the old index commits will be cleaned up as soon as possible. Follow-up of #28140 #28140 (comment)
We currently revisit the index deletion policy whenever the global checkpoint has advanced enough. We should also revisit the deletion policy after releasing the last snapshot of a snapshotting commit. With this change, the old index commits will be cleaned up as soon as possible. Follow-up of #28140 #28140 (comment)
Since elastic#28140 when the global checkpoint is advanced, we try to move the safe commit forward, and clean old index commits if possible. However, we forget to trim unreferenced translog. This change makes sure that we prune both translog and index commits when the safe commit advanced. Relates elastic#28140 Closes elastic#32089
Since #28140 when the global checkpoint is advanced, we try to move the safe commit forward, and clean up old index commits if possible. However, we forget to trim unreferenced translog. This change makes sure that we prune both old translog and index commits when the safe commit advanced. Relates #28140 Closes #32089
Since #28140 when the global checkpoint is advanced, we try to move the safe commit forward, and clean up old index commits if possible. However, we forget to trim unreferenced translog. This change makes sure that we prune both old translog and index commits when the safe commit advanced. Relates #28140 Closes #32089
Since #28140 when the global checkpoint is advanced, we try to move the safe commit forward, and clean up old index commits if possible. However, we forget to trim unreferenced translog. This change makes sure that we prune both old translog and index commits when the safe commit advanced. Relates #28140 Closes #32089
Since #28140 when the global checkpoint is advanced, we try to move the safe commit forward, and clean up old index commits if possible. However, we forget to trim unreferenced translog. This change makes sure that we prune both old translog and index commits when the safe commit advanced. Relates #28140 Closes #32089
Since #28140 when the global checkpoint is advanced, we try to move the safe commit forward, and clean up old index commits if possible. However, we forget to trim unreferenced translog. This change makes sure that we prune both old translog and index commits when the safe commit advanced. Relates #28140 Closes #32089
Today we keep multiple index commits based on the global checkpoint, but only clean up old index commits when we have a new index commit. However, we can release unneeded index commits earlier once the global checkpoint has advanced enough. This commit makes an engine revisit the index deletion policy whenever a new global checkpoint value is persisted.
Relates #10708