Skip to content

RegionMigrate: Fix migrating region with ratisConsensus cause the region is unavailable.#13178

Merged
OneSizeFitsQuorum merged 19 commits intoapache:masterfrom
133tosakarin:fix_reset_peer_fail
Aug 19, 2024
Merged

RegionMigrate: Fix migrating region with ratisConsensus cause the region is unavailable.#13178
OneSizeFitsQuorum merged 19 commits intoapache:masterfrom
133tosakarin:fix_reset_peer_fail

Conversation

@133tosakarin
Copy link
Contributor

@133tosakarin 133tosakarin commented Aug 15, 2024

If the current region configuration is [a, b, c], we execute the migrate region id from c to d, and the leader is a. During the region migration process, due to the need to catch up with the new node d, if the data is large, it may take a long time.

Assuming that the operations before the execution of the Remove phase are successful, the region configuration at this time should be [a, b, c, d], and then start executing RemoveRegion, but due to some reasons, the RemoveRegion operation has not been completed, resulting in the failure of the DELETE_REGION_PEER phase timeout in the executeFromState function of RemoveProcessure, and then enter the DELETE_OLD_REGION_PEER, the new node d has not caught up with a successfully, and the old node c has been deleted.

If the leaderTransfership is executed midway, such as a to b or a to d, then the entire region group will be in a state without a leader, which is permanent because c thinks it is not in the region group, but other nodes [a, b, d] It is believed that if it is in the region group and the new node d has not caught up with the old leader, d will not execute the vote (due to the lifeCycle of ratis), then this region group will never be elected as the leader because the candidate will not receive the majority vote.

We modify the member change part and wait for the raft member change to complete or throw an exception before returning

In the previous region migration implementation, we randomly selected an available node a and relinquished the existing leader privileges to this available node a. This may result in selecting a new node whose data has not yet caught up with the previous leader. This useless election is wasted because other nodes will never vote for node a.
In this PR, we fixed this issue, and if the migrated node is not the leader, then we do not perform the leader transfer

@133tosakarin 133tosakarin changed the title RegionMigrate: Fix choosing the transferLeadership on migrateRegion RegionMigrate: Fix migrating region cause the region is unavailable. Aug 15, 2024
@133tosakarin 133tosakarin changed the title RegionMigrate: Fix migrating region cause the region is unavailable. RegionMigrate: Fix migrating region with ratisConsensus cause the region is unavailable. Aug 15, 2024
Integer leaderId = configManager.getLoadManager().getRegionLeaderMap().get(regionId);

if (leaderId != -1) {
// The migrated node is not leader, so we specify the transfer leader to current leader node
Copy link
Contributor

Choose a reason for hiding this comment

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

then we do not need to send rpc to datanode?

Copy link
Contributor Author

@133tosakarin 133tosakarin Aug 15, 2024

Choose a reason for hiding this comment

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

I think so, we can report success

status);
configManager.getLoadManager().removeRegionCache(regionId, deprecatedLocation.getDataNodeId());
configManager.getLoadManager().getRouteBalancer().balanceRegionLeaderAndPriority();
// configManager.getLoadManager().getRouteBalancer().balanceRegionLeaderAndPriority();
Copy link
Contributor

Choose a reason for hiding this comment

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

do not comment this

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok, i have changed it

*/
public void transferRegionLeader(TConsensusGroupId regionId, TDataNodeLocation originalDataNode)
throws ProcedureException, InterruptedException {
List<TDataNodeLocation> excludeDataNode = new ArrayList<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

Collections.singletonList()

Copy link
Contributor Author

Choose a reason for hiding this comment

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

:), ok

public Optional<TDataNodeLocation> filterDataNodeWithOtherRegionReplica(
TConsensusGroupId regionId, TDataNodeLocation filterLocation, NodeStatus... allowingStatus) {
List<TDataNodeLocation> excludeLocations = new ArrayList<>();
excludeLocations.add(filterLocation);
Copy link
Contributor

Choose a reason for hiding this comment

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

same as above

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 have changed

regionId,
new ConsensusGroupHeartbeatSample(timestamp, newLeaderNode.get().getDataNodeId())));
configManager.getLoadManager().getRouteBalancer().balanceRegionLeaderAndPriority();
// configManager.getLoadManager().getRouteBalancer().balanceRegionLeaderAndPriority();
Copy link
Contributor

Choose a reason for hiding this comment

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

do not comment

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok

Optional.ofNullable(getGroupInfo(raftGroupId))
.orElseThrow(() -> new ConsensusGroupNotExistException(groupId));

if (raftGroup.getPeers() == null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

if we forbid leader trasnfer during regionMigration, thus we do not need this judgement?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This commit is draft version, I'll delete these code in future commit

Copy link
Contributor

Choose a reason for hiding this comment

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

OK, waiting for yuheng's pr

Copy link
Contributor

Choose a reason for hiding this comment

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

remove it

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok


final RaftClientReply reply;
try {
Peer leader = getLeader(groupId);
Copy link
Contributor

Choose a reason for hiding this comment

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

same as above, not necessary if we consider this in condignode

return reply;
}

private RaftClientReply sendReconfigurationWithRetry(RaftGroup newGroupConf)
Copy link
Contributor

Choose a reason for hiding this comment

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

not used?

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 have used another policy

// The two fields are used to control the retry times and wait time for setConfiguration
// reconfiguration may take many time, so we need to set a long wait time and retry times
private final long setConfigurationWaitTime = 10000;
private final int setConfigurationRetryTimes = 50;
Copy link
Contributor

Choose a reason for hiding this comment

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

not used?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

these codes have been deleted

this.config.getImpl().getRetryWaitMillis(), TimeUnit.MILLISECONDS))
.build();

regionMigrateRetryPolicy =
Copy link
Contributor

Choose a reason for hiding this comment

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

not used?

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'm considering which one to use

handler.forceUpdateRegionCache(consensusGroupId, targetDataNode, RegionStatus.Removing);
List<TDataNodeLocation> excludeDataNode = new ArrayList<>();
excludeDataNode.add(targetDataNode);
excludeDataNode.add(coordinator);
Copy link
Contributor

Choose a reason for hiding this comment

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

It feels like the semantics here should be allowed, and in the end, if the judgment is the same, do not send rpc?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This just exclude origin node and target node, we don't need to transferLeadership to a new member, and the originNode will be deleted, we also filter it out. Considering that there may be other nodes that need to be filtered, I simply used List to avoid adding additional parameters to the function

*/
public Optional<TDataNodeLocation> filterDataNodeWithOtherRegionReplica(
TConsensusGroupId regionId, TDataNodeLocation filterLocation) {
List<TDataNodeLocation> filterLocations = new ArrayList<>();
Copy link
Contributor

Choose a reason for hiding this comment

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

Collections.singletonList

Copy link
Contributor

Choose a reason for hiding this comment

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

why not change?

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 didn't see it

public void transferRegionLeader(
TConsensusGroupId regionId,
TDataNodeLocation originalDataNode,
List<TDataNodeLocation> excludeDataNode)
Copy link
Contributor

Choose a reason for hiding this comment

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

seems we only need one node which will be deleted in the future? not a list?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It's convenient to use to determine whether an object is included

Optional.ofNullable(getGroupInfo(raftGroupId))
.orElseThrow(() -> new ConsensusGroupNotExistException(groupId));

if (raftGroup.getPeers() == null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

OK, waiting for yuheng's pr

client.getRaftClient().admin().setConfiguration(new ArrayList<>(newGroupConf.getPeers()));
if (!reply.isSuccess()) {
int basicWaitTime = 500;
int maxWaitTime = 10000;
Copy link
Contributor

Choose a reason for hiding this comment

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

It feels like we can just check regularly here, without adding idempotent logic, because it doesn't make a big difference.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is also okay.

break;
}
if (reply.getException() instanceof ReconfigurationInProgressException
|| reply.getException() instanceof LeaderSteppingDownException
Copy link
Contributor

Choose a reason for hiding this comment

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

when we need thses two exceptions? will this cause forever loop?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it will loop until reply return success or throw other exception. In fact, I want to use RaftServer.LifeCycle as loop factor, if majority RaftServer.LifeCycle is RUNNING, then we can return, but I can't get raftServers in newGroupConf's peers

Copy link
Collaborator

@liyuheng55555 liyuheng55555 left a comment

Choose a reason for hiding this comment

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

Please take a look.

Besides, please break down the first paragraph of the PR description into smaller paragraphs to improve readability.

Comment on lines +85 to +88
List<TDataNodeLocation> excludeDataNode = new ArrayList<>();
excludeDataNode.add(targetDataNode);
excludeDataNode.add(coordinator);
handler.transferRegionLeader(consensusGroupId, targetDataNode, excludeDataNode);

This comment was marked as resolved.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Then I'll change it again.


static {
String str = "";
// 50, 500ms, 40, 1000ms, 30, 1500ms, 20, 2000ms, 10, 2500ms
Copy link
Collaborator

Choose a reason for hiding this comment

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

What is this comment's meaning? Describe your algorithm more clearly.

And, why define an algorithm in static chunk?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it's a pairArray, pair such as (n, t), use the i-th pair, then client will retry n times, each time sleep t ms.

I cancat these pair as str, The MultipleLinearRandomRetry will parse it.

I want to calculate only once, so I think static chunk will only be executed once

Copy link
Collaborator

Choose a reason for hiding this comment

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

I think just using "50, 500ms, 40, 1000ms, 30, 1500ms, 20, 2000ms, 10, 2500ms" will be more clear than building a String by loop.

But, this strategy seems not actually endless ?

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, the retry time should give a large number.
if the number is enough large, then the loop is ok

Comment on lines +239 to +246
// Ratis guarantees that event.getCause() is instance of IOException.
// We should allow RaftException or IOException(StatusRuntimeException, thrown by gRPC) to be
// retried.
Optional<Throwable> unexpectedCause =
Optional.ofNullable(event.getCause())
.filter(RaftException.class::isInstance)
.map(Throwable::getCause)
.filter(StatusRuntimeException.class::isInstance);
Copy link
Collaborator

Choose a reason for hiding this comment

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

What is RaftException or IOException(StatusRuntimeException, thrown by gRPC)? Words between () is confusing, StatusRuntimeException is not a kind of IOException.

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 copy from RatisRetryPolicy

private final RaftClientRpc clientRpc;

private final IClientManager<RaftGroup, RatisClient> clientManager;
private final IClientManager<RaftGroup, RatisClient> reconfClientManager;

This comment was marked as resolved.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok:)

}
}

private RatisClient getConfRaftClient(RaftGroup group) throws ClientManagerException {

This comment was marked as resolved.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok:)

Comment on lines +925 to +929
private boolean isReConfiguration;

RatisClientPoolFactory(boolean isReConfiguration) {
this.isReConfiguration = isReConfiguration;
}
Copy link
Collaborator

Choose a reason for hiding this comment

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

isReconfiguration

Copy link
Contributor Author

Choose a reason for hiding this comment

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

what's mean?

Copy link
Collaborator

Choose a reason for hiding this comment

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

C->c


@Override
public Action handleAttemptFailure(Event event) {
if (event.getCause() instanceof ReconfigurationInProgressException) {
Copy link
Contributor

Choose a reason for hiding this comment

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

why not use if(A || B || C)

Copy link
Contributor Author

Choose a reason for hiding this comment

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

haha! I'm lazy here :) . This part is the copilot prompt, and the tab is generated with one click.

Copy link
Contributor

Choose a reason for hiding this comment

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

Why not change this?

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 have changed

Comment on lines +684 to +687
if (!newLeaderNode.isPresent()) {
// If we have no choice, we use it
newLeaderNode = Optional.of(coodinator);
}
Copy link
Collaborator

Choose a reason for hiding this comment

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

Better than my thought, nice and clean code!

Comment on lines +220 to +221
// This policy is used to raft configuration change
//
Copy link
Collaborator

Choose a reason for hiding this comment

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

Use javadoc /** */ 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.

ok

Comment on lines +927 to +928
RatisClientPoolFactory(boolean isReConfiguration) {
this.isReConfiguration = isReConfiguration;
this.isReconfiguration = isReConfiguration;
Copy link
Collaborator

Choose a reason for hiding this comment

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

you forget something

Comment on lines +227 to 231
RatisEndlessRetryPolicy() {
// about 1 hour wait Time.
defaultPolicy =
MultipleLinearRandomRetry.parseCommaSeparated(str.substring(0, str.length() - 1));
RetryPolicies.retryForeverWithSleep(TimeDuration.valueOf(2, TimeUnit.SECONDS));
}
Copy link
Collaborator

Choose a reason for hiding this comment

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

  1. Isn't retry every 2 seconds too frequently ?
  2. Why still // about 1 hour wait Time.

Copy link
Contributor Author

@133tosakarin 133tosakarin Aug 16, 2024

Choose a reason for hiding this comment

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

  1. if reply return false, then resend request after 2s. Instead of sending it every 2 seconds

  2. sorry, I forget delete comment


private RatisClient getConfigurationRaftClient(RaftGroup group) throws ClientManagerException {
try {
return reconfigurationClientManager.borrowClient(group);
Copy link
Contributor

Choose a reason for hiding this comment

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

consistent with clientManager

Optional.ofNullable(getGroupInfo(raftGroupId))
.orElseThrow(() -> new ConsensusGroupNotExistException(groupId));

if (raftGroup.getPeers() == null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

remove it

excludeDataNode.add(coodinator);
while (System.nanoTime() - startTime < TimeUnit.SECONDS.toNanos(findNewLeaderTimeLimitSecond)) {
newLeaderNode = filterDataNodeWithOtherRegionReplica(regionId, originalDataNode);
newLeaderNode = filterDataNodeWithOtherRegionReplica(regionId, excludeDataNode);
Copy link
Contributor

Choose a reason for hiding this comment

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

cause redundant sleep when replica is 1?

Copy link
Collaborator

@liyuheng55555 liyuheng55555 left a comment

Choose a reason for hiding this comment

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

LGTM

@OneSizeFitsQuorum OneSizeFitsQuorum merged commit 45ab21b into apache:master Aug 19, 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.

3 participants