You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@iotdb.apache.org by "Xinyu Tan (Jira)" <ji...@apache.org> on 2021/08/16 14:26:00 UTC

[jira] [Commented] (IOTDB-1564) [Distributed] Make leader failure detection and election faster

    [ https://issues.apache.org/jira/browse/IOTDB-1564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17399768#comment-17399768 ] 

Xinyu Tan commented on IOTDB-1564:
----------------------------------

This is a meaningful discussion because timeout parameters in cluster can indeed be more refined.

I am looking  forward to your participation in the cluster kernel development! Below, I will introduce my views according to your chapter

A)I think two arguments are really enough: heartbeat_timeout_ms and election_timeout_ms. These are the two parameters that must be considered in any Raft implementation.

For heartbeat_timeout_ms, this parameter can be used as the time at which the leader sends heartbeats. This parameter is currently hard-coded in the code and can be made into a configurable parameter.

For election_timeout_ms, this parameter can be used as the election timeout time of candidates and followers, or as the parameter of waiting for voting results. As for the latter, since the steps of sending RPC are asynchronous, the time of other operations in startElections can be basically negligible, so such semantics have been consistent with some Raft corner cases, such as split vote scenarios. The candidate must call the election again after election timeout. Personally, I don't think it's necessary to make  `The max time waiting for the vote response` as a parameter, just use election_timeout_ms is enough.

B) Couldn't agree more

C) Good idea, but it seems to have been implemented. See line 266 of class HeartBeatThread for details. Raft uses random election timeout to avoid livelocking, so we need to include some mechanism to avoid livelocking as well.

As for the randomTime function you mentioned to implement something like load balancing, I think a more direct way is to directly implement Raft's transferLeader and let the idle node directly perform an election, which may be more direct and effective. If you are interested in implementing this feature, you can refer to etcd, and you are welcome to discuss the design with us.

> [Distributed] Make leader failure detection and election faster
> ---------------------------------------------------------------
>
>                 Key: IOTDB-1564
>                 URL: https://issues.apache.org/jira/browse/IOTDB-1564
>             Project: Apache IoTDB
>          Issue Type: Improvement
>          Components: Cluster
>            Reporter: Eric Pai
>            Assignee: Eric Pai
>            Priority: Major
>             Fix For: master branch
>
>
> The cluster configuration _connection_timeout_ms_ is now used in different layers:
> 1. The connection and socket timeout of underlying TSocket of the Thrift framework.
>  2. The CatchUpTask.
>  *3. The heartbeat expired time of RaftMember.*
>  *4. The sleep time between adjcent FOLLOWER heartbeat timeout validations.*
>  *5. The election timeout.*
> However, it doesn't make sense that those time must be same. A longer heartbeat expired time means a delayed detection of the leader failure. Thus we should separate it as a new configuration parameter for DBAs.
> Except for the network latency, +4+ and +5+ are the major impactions of the time cost from leader failure to a successful election ends. We can do some optimizations for them. Here are my solutions:
> a) Add new cluster configurations: _heartbeat_timeout_ms, heartbeat_interval_ms_ and _election_timeout_ms_, and leave _connection_timeout_ms_ as the TSocket timeout only (this also satisfies the literal meaning).
>  * _heartbeat_interval_ms_: The time interval between two rounds of heartbeat broadcast of one raft group leader
>  * _heartbeat_timeout_ms_: The max expired time from _lastHeartbeatReceivedTime_. If current time exceeds _heartbeat_timeout_ms_ + _lastHeartbeatReceivedTime_, a new election starts.
>  * _election_timeout_ms_: The max time waiting for the vote response.
> b) We can also make +4+ process more wizardly.
> Because we already know the _lastHeartbeatReceivedTime_, then the expired time of this heartbeat is _lastHeartbeatReceivedTime_ + _heartbeat_timeout_ms_.
> Thus we can sleep _lastHeartbeatReceivedTime_ + _heartbeat_timeout_ms_ - _now()_ for the next check. If the heartbeat timeout happens, RaftMember will start election immediately.
> c) If multi RaftMembers start election at the sametime, all elections may fail because of receiving insufficent votes, and every member will wait a random time for next election, which increases the whole election duration. In order to improve the election successful rate, we can add a random delayed time to +b+:
>  - Sleep: _lastHeartbeatReceivedTime + heartbeat_timeout_ms - now() + randomTime()_
> The less value _randomTime()_ a member gets, the higher probability it has to become the new leader.
> This design has another benfit. We know that the leader node has heavier workload than followers. In the future we can make the idlest node be the leader during election, by calculating the resource usage and returing a lower value from the randomTime() method in the idle node.
> Any suggestions or disscussions are welcomed :)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)