Skip to content

Commit

Permalink
RATIS-1918 Introduces parameters that separately control the shutdown…
Browse files Browse the repository at this point in the history
… of RaftServerProxy by JVMPauseMonitor. (apache#950)
  • Loading branch information
OneSizeFitsQuorum authored Oct 24, 2023
1 parent 94a8fbe commit a0f74e5
Show file tree
Hide file tree
Showing 2 changed files with 12 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,16 @@ static void setSleepDeviationThreshold(RaftProperties properties, int thresholdM
setInt(properties::setInt, SLEEP_DEVIATION_THRESHOLD_KEY, thresholdMs);
}

String CLOSE_THRESHOLD_KEY = PREFIX + ".close.threshold";
TimeDuration CLOSE_THRESHOLD_DEFAULT = TimeDuration.valueOf(60, TimeUnit.SECONDS);
static TimeDuration closeThreshold(RaftProperties properties) {
return getTimeDuration(properties.getTimeDuration(CLOSE_THRESHOLD_DEFAULT.getUnit()),
CLOSE_THRESHOLD_KEY, CLOSE_THRESHOLD_DEFAULT, getDefaultLog());
}
static void setCloseThreshold(RaftProperties properties, int thresholdMs) {
setInt(properties::setInt, CLOSE_THRESHOLD_KEY, thresholdMs);
}

/**
* When bootstrapping a new peer, If the gap between the match index of the
* peer and the leader's latest committed index is less than this gap, we
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -222,11 +222,11 @@ String toString(RaftGroupId groupId, CompletableFuture<RaftServerImpl> f) {
id + "-impl"));

final TimeDuration sleepDeviationThreshold = RaftServerConfigKeys.sleepDeviationThreshold(properties);
final TimeDuration rpcSlownessTimeout = RaftServerConfigKeys.Rpc.slownessTimeout(properties);
final TimeDuration closeThreshold = RaftServerConfigKeys.closeThreshold(properties);
final TimeDuration leaderStepDownWaitTime = RaftServerConfigKeys.LeaderElection.leaderStepDownWaitTime(properties);
this.pauseMonitor = JvmPauseMonitor.newBuilder().setName(id)
.setSleepDeviationThreshold(sleepDeviationThreshold)
.setHandler(extraSleep -> handleJvmPause(extraSleep, rpcSlownessTimeout, leaderStepDownWaitTime))
.setHandler(extraSleep -> handleJvmPause(extraSleep, closeThreshold, leaderStepDownWaitTime))
.build();
this.threadGroup = threadGroup == null ? new ThreadGroup(this.id.toString()) : threadGroup;
}
Expand Down

0 comments on commit a0f74e5

Please sign in to comment.