|
|
@@ -176,6 +176,16 @@ type Config struct {
|
|
|
// Logger is the logger used for raft log. For multinode which can host
|
|
|
// multiple raft group, each raft group can have its own logger
|
|
|
Logger Logger
|
|
|
+
|
|
|
+ // DisableProposalForwarding set to true means that followers will drop
|
|
|
+ // proposals, rather than forwarding them to the leader. One use case for
|
|
|
+ // this feature would be in a situation where the Raft leader is used to
|
|
|
+ // compute the data of a proposal, for example, adding a timestamp from a
|
|
|
+ // hybrid logical clock to data in a monotonically increasing way. Forwarding
|
|
|
+ // should be disabled to prevent a follower with an innaccurate hybrid
|
|
|
+ // logical clock from assigning the timestamp and then forwarding the data
|
|
|
+ // to the leader.
|
|
|
+ DisableProposalForwarding bool
|
|
|
}
|
|
|
|
|
|
func (c *Config) validate() error {
|
|
|
@@ -256,6 +266,7 @@ type raft struct {
|
|
|
// [electiontimeout, 2 * electiontimeout - 1]. It gets reset
|
|
|
// when raft changes its state to follower or candidate.
|
|
|
randomizedElectionTimeout int
|
|
|
+ disableProposalForwarding bool
|
|
|
|
|
|
tick func()
|
|
|
step stepFunc
|
|
|
@@ -283,18 +294,19 @@ func newRaft(c *Config) *raft {
|
|
|
peers = cs.Nodes
|
|
|
}
|
|
|
r := &raft{
|
|
|
- id: c.ID,
|
|
|
- lead: None,
|
|
|
- raftLog: raftlog,
|
|
|
- maxMsgSize: c.MaxSizePerMsg,
|
|
|
- maxInflight: c.MaxInflightMsgs,
|
|
|
- prs: make(map[uint64]*Progress),
|
|
|
- electionTimeout: c.ElectionTick,
|
|
|
- heartbeatTimeout: c.HeartbeatTick,
|
|
|
- logger: c.Logger,
|
|
|
- checkQuorum: c.CheckQuorum,
|
|
|
- preVote: c.PreVote,
|
|
|
- readOnly: newReadOnly(c.ReadOnlyOption),
|
|
|
+ id: c.ID,
|
|
|
+ lead: None,
|
|
|
+ raftLog: raftlog,
|
|
|
+ maxMsgSize: c.MaxSizePerMsg,
|
|
|
+ maxInflight: c.MaxInflightMsgs,
|
|
|
+ prs: make(map[uint64]*Progress),
|
|
|
+ electionTimeout: c.ElectionTick,
|
|
|
+ heartbeatTimeout: c.HeartbeatTick,
|
|
|
+ logger: c.Logger,
|
|
|
+ checkQuorum: c.CheckQuorum,
|
|
|
+ preVote: c.PreVote,
|
|
|
+ readOnly: newReadOnly(c.ReadOnlyOption),
|
|
|
+ disableProposalForwarding: c.DisableProposalForwarding,
|
|
|
}
|
|
|
for _, p := range peers {
|
|
|
r.prs[p] = &Progress{Next: 1, ins: newInflights(r.maxInflight)}
|
|
|
@@ -1033,6 +1045,9 @@ func stepFollower(r *raft, m pb.Message) {
|
|
|
if r.lead == None {
|
|
|
r.logger.Infof("%x no leader at term %d; dropping proposal", r.id, r.Term)
|
|
|
return
|
|
|
+ } else if r.disableProposalForwarding {
|
|
|
+ r.logger.Infof("%x not forwarding to leader %x at term %d; dropping proposal", r.id, r.lead, r.Term)
|
|
|
+ return
|
|
|
}
|
|
|
m.To = r.lead
|
|
|
r.send(m)
|