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

RATIS-1390. Bootstrapping Peer should always try to install a snapshot the first time. #489

Merged
merged 4 commits into from
Aug 22, 2021

Conversation

hanishakoneru
Copy link
Contributor

What changes were proposed in this pull request?

When a peer is bootstrapping and joins the Ratis ring via SetConf request, the Leader adds the peer to the sender list and sends it either the append log entries or the snapshot or a notification to install snapshot via State Machine. The leader sends the snapshot only when some or all of the ratis logs have been purged.

The snapshot is maintained by the State Machine and as such could contain information not present in the Ratis logs (for example Ozone's OzoneManager StateMachine maintains the snapshot as a DB file and it could contain information which cannot be constructed just using the Ratis logs - HDDS-5338).

To avoid this scenario, this Jira proposes that whenever a new peer is bootstrapping, the leader should send an install snapshot request at least once.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/RATIS-1390

How was this patch tested?

Will add a unit test in next iteration.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@hanishakoneru , thanks a lot for working on this. Two minor comments about the names.

BTW, should we also add a new test?

@@ -52,6 +52,14 @@
/** Set follower's snapshotIndex. */
void setSnapshotIndex(long newSnapshotIndex);

/** Acknowledge that Follower attempted to install a snapshot. It does not guarantee that the installation was
* successful. This helps to determine whether Follower can come out of bootstrap process. */
void ackInstallSnapshotAttempt();
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's use a name similar to hasAttemptedToInstallSnapshot(). How about "setAttemptedToInstallSnapshot"?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

@@ -148,6 +148,7 @@ enum InstallSnapshotResult {
IN_PROGRESS = 2;
ALREADY_INSTALLED = 3;
CONF_MISMATCH = 4;
NULL_SNAPSHOT = 5;
Copy link
Contributor

Choose a reason for hiding this comment

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

How about calling it as SNAPSHOT_UNAVALIABLE?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure. Updated.

@hanishakoneru
Copy link
Contributor Author

Thanks @szetszwo for the review. I have addressed your comments and added 2 unit tests.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

The change looks good. Thanks for the update.

TestInstallSnapshotNotificationWithGrpc failed. Could you take a look?

@hanishakoneru
Copy link
Contributor Author

The failing tests look unrelated and pass locally. I have retriggered CI.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

+1 the change looks good.

@szetszwo szetszwo merged commit cc5f2e5 into apache:master Aug 22, 2021
symious pushed a commit to symious/ratis that referenced this pull request Feb 20, 2024
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