Skip to content

Fix Snapshot Out of Order Finalization Repo Corruption#75362

Merged
original-brownbear merged 14 commits intoelastic:masterfrom
original-brownbear:repro-multiple-out-of-ordersnapshot-finalization
Jul 16, 2021
Merged

Fix Snapshot Out of Order Finalization Repo Corruption#75362
original-brownbear merged 14 commits intoelastic:masterfrom
original-brownbear:repro-multiple-out-of-ordersnapshot-finalization

Conversation

@original-brownbear
Copy link
Copy Markdown
Contributor

@original-brownbear original-brownbear commented Jul 15, 2021

  • Fix up shard generations in SnapshotsInProgress during snapshot finalization (don't do it earlier because it's a really heavy computation and we have a ton of places where it would have to run).
  • Adjust finalization queue to be able to work with changing snapshot entries after they've been enqueued for finalisation
  • Still one remaining bug left after this (see TODO about leaking generations) that I don't feel confident in fixing for 7.13.4 due to the complexity of a fix and how minor the blob leak is (+ it's cleaned up just fine during snapshot deletes)

NOTE: this could probably be dried up a lot against other shard state machine logic but I did want to isolate this change as much as I could for easy backporting as well as to minimise risk as it's not a trivial change at all. By only running the generation fixing after finalizing and with the newly added tests I feel confident in this fix though.

Closes #75336

@original-brownbear original-brownbear added >bug WIP :Distributed/Snapshot/Restore Anything directly related to the `_snapshot/*` APIs labels Jul 15, 2021
@original-brownbear original-brownbear marked this pull request as ready for review July 15, 2021 13:45
@elasticmachine elasticmachine added the Team:Distributed Meta label for distributed team. label Jul 15, 2021
@elasticmachine
Copy link
Copy Markdown
Collaborator

Pinging @elastic/es-distributed (Team:Distributed)

*/
private static ClusterState stateWithoutSnapshot(ClusterState state, Snapshot snapshot) {
private static ClusterState stateWithoutSuccessfulSnapshot(ClusterState state, Snapshot snapshot) {
// TODO: updating snapshots here leaks their outdated generation files, we should add logic to clean those up and enhance
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

The logic in this method would definitely benefit from a state tracking object like we have for the shard status update executor. In the interest of time and keeping it simple I went with this solution for now. Ideally, I'd like to resolve this todo and make the whole logic simpler to follow in 7.15 by finally refactoring SnapshotsInProgress into a form that is more appropriate for the logic around concurrent snapshots than to add yet another round of elaborate logic to nicely work around its shortcomings.

@original-brownbear
Copy link
Copy Markdown
Contributor Author

Jenkins run elasticsearch-ci/part-1 (unrelated + known)

@original-brownbear
Copy link
Copy Markdown
Contributor Author

Jenkins run elasticsearch-ci/part-2 (unrelated + known)

1 similar comment
@original-brownbear
Copy link
Copy Markdown
Contributor Author

Jenkins run elasticsearch-ci/part-2 (unrelated + known)

final String bestGeneration = generations.getOrDefault(indexName, Collections.emptyMap()).get(shardId);
assert bestGeneration == null || activeGeneration == null || activeGeneration.equals(bestGeneration);
if ((bestGeneration == null || activeGeneration == null || activeGeneration.equals(bestGeneration)) == false) {
throw new AssertionFailedException("gnarf");
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Changing this assertion failure into a runtime exception is, I think, cheating ;)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

sorry needed a breakpoint there :D

);
}

public void testOutOfOrderCloneFinalization() throws Exception {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

I think the clone - clone case is missing?

Copy link
Copy Markdown
Contributor Author

@original-brownbear original-brownbear Jul 16, 2021

Choose a reason for hiding this comment

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

Yea sorta, that's incredibly hard to reproduce because it all runs on the same master node (can't selectively block one just one shard easily with the current infra) and I couldn't find a quick way of adding the infrastructure for that test. I can try to find time for it later today but no guarantees I will be able to.

Copy link
Copy Markdown
Contributor

@fcofdez fcofdez left a comment

Choose a reason for hiding this comment

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

LGTM, but I agree that we should simplify this logic in the near future, it's becoming quite complex to follow. 👍

Copy link
Copy Markdown
Member

@DaveCTurner DaveCTurner left a comment

Choose a reason for hiding this comment

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

I left one question and one tiny nit, LGTM otherwise.

ImmutableOpenMap.Builder<RepositoryShardId, ShardSnapshotStatus> updatedShardAssignments = null;
for (ObjectObjectCursor<RepositoryShardId, ShardSnapshotStatus> finishedShardEntry : removedEntry.clones()) {
final ShardSnapshotStatus shardState = finishedShardEntry.value;
if (shardState.state() == ShardState.SUCCESS) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

tiny nit: we're inconsistent about this vs if (shardState.state() != ShardState.SUCCESS) { continue; } across the 4 branches

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Ah I did this on purpose to not indent so deeply when there's more complicated logic in a branch ... maybe just confusing though I can change it if you want :)

final RepositoryShardId repoShardId = finishedShardEntry.key;
final IndexMetadata indexMeta = state.metadata().index(repoShardId.indexName());
if (indexMeta == null) {
// The index name that finished cloning does not exist in the cluster state so it isn't relevant
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

I'm confused by this. If we deleted this index and then created another one with the same name then we'd be updating the entry with the wrong index UUID. I'm not sure this matters, but it certainly seems like it puts the entry in a strange state. Can we not use the shard ID from the actual entry?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Actually in this case org.elasticsearch.snapshots.SnapshotsService#maybeAddUpdatedAssignment will just not find the shard entry in the snapshot's map that still contains the old uuid because the ShardId won't be equal so that should be fine I think.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

ugh ok even weirder :) Looking forward to this all getting cleaned up soon...

Copy link
Copy Markdown
Member

@DaveCTurner DaveCTurner left a comment

Choose a reason for hiding this comment

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

LGTM

@original-brownbear
Copy link
Copy Markdown
Contributor Author

Thanks David + Francisco!

@original-brownbear original-brownbear merged commit 3bd2672 into elastic:master Jul 16, 2021
@original-brownbear original-brownbear deleted the repro-multiple-out-of-ordersnapshot-finalization branch July 16, 2021 12:51
original-brownbear added a commit that referenced this pull request Jul 16, 2021
* Fix up shard generations in `SnapshotsInProgress` during snapshot finalization (don't do it earlier because it's a really heavy computation and we have a ton of places where it would have to run).
* Adjust finalization queue to be able to work with changing snapshot entries after they've been enqueued for finalisation
* Still one remaining bug left after this (see TODO about leaking generations) that I don't feel confident in fixing for `7.13.4` due to the complexity of a fix and how minor the blob leak is (+ it's cleaned up just fine during snapshot deletes)

Closes #75336
original-brownbear added a commit that referenced this pull request Jul 16, 2021
* Fix up shard generations in `SnapshotsInProgress` during snapshot finalization (don't do it earlier because it's a really heavy computation and we have a ton of places where it would have to run).
* Adjust finalization queue to be able to work with changing snapshot entries after they've been enqueued for finalisation
* Still one remaining bug left after this (see TODO about leaking generations) that I don't feel confident in fixing for `7.13.4` due to the complexity of a fix and how minor the blob leak is (+ it's cleaned up just fine during snapshot deletes)

Closes #75336
ywangd pushed a commit to ywangd/elasticsearch that referenced this pull request Jul 30, 2021
* Fix up shard generations in `SnapshotsInProgress` during snapshot finalization (don't do it earlier because it's a really heavy computation and we have a ton of places where it would have to run).
* Adjust finalization queue to be able to work with changing snapshot entries after they've been enqueued for finalisation
* Still one remaining bug left after this (see TODO about leaking generations) that I don't feel confident in fixing for `7.13.4` due to the complexity of a fix and how minor the blob leak is (+ it's cleaned up just fine during snapshot deletes)

Closes elastic#75336
@original-brownbear original-brownbear restored the repro-multiple-out-of-ordersnapshot-finalization branch April 18, 2023 20:55
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

>bug :Distributed/Snapshot/Restore Anything directly related to the `_snapshot/*` APIs Team:Distributed Meta label for distributed team. v7.14.0 v7.15.0 v8.0.0-alpha1

Projects

None yet

Development

Successfully merging this pull request may close these issues.

Concurrent Snapshots Finalizing out of Order May Corrupt a Repository

6 participants