@@ -283,6 +283,13 @@ type Config struct {
283283 // This behavior will become unconditional in the future. See:
284284 // https://github.com/etcd-io/raft/issues/83
285285 StepDownOnRemoval bool
286+
287+ // DisableProposalForwardingCallback will be called for each MsgProp message
288+ // on nodes which are in follower state.
289+ // If this callback returns true, the message will be discarded.
290+ // This callback function is used for implementing a mechanism like
291+ // DisableProposalForwarding for each message instead of global configuration.
292+ DisableProposalForwardingCallback func (m pb.Message ) bool
286293}
287294
288295func (c * Config ) validate () error {
@@ -413,9 +420,10 @@ type raft struct {
413420 // randomizedElectionTimeout is a random number between
414421 // [electiontimeout, 2 * electiontimeout - 1]. It gets reset
415422 // when raft changes its state to follower or candidate.
416- randomizedElectionTimeout int
417- disableProposalForwarding bool
418- stepDownOnRemoval bool
423+ randomizedElectionTimeout int
424+ disableProposalForwarding bool
425+ stepDownOnRemoval bool
426+ disableProposalForwardingCallback func (m pb.Message ) bool
419427
420428 tick func ()
421429 step stepFunc
@@ -440,22 +448,23 @@ func newRaft(c *Config) *raft {
440448 }
441449
442450 r := & raft {
443- id : c .ID ,
444- lead : None ,
445- isLearner : false ,
446- raftLog : raftlog ,
447- maxMsgSize : entryEncodingSize (c .MaxSizePerMsg ),
448- maxUncommittedSize : entryPayloadSize (c .MaxUncommittedEntriesSize ),
449- prs : tracker .MakeProgressTracker (c .MaxInflightMsgs , c .MaxInflightBytes ),
450- electionTimeout : c .ElectionTick ,
451- heartbeatTimeout : c .HeartbeatTick ,
452- logger : c .Logger ,
453- checkQuorum : c .CheckQuorum ,
454- preVote : c .PreVote ,
455- readOnly : newReadOnly (c .ReadOnlyOption ),
456- disableProposalForwarding : c .DisableProposalForwarding ,
457- disableConfChangeValidation : c .DisableConfChangeValidation ,
458- stepDownOnRemoval : c .StepDownOnRemoval ,
451+ id : c .ID ,
452+ lead : None ,
453+ isLearner : false ,
454+ raftLog : raftlog ,
455+ maxMsgSize : entryEncodingSize (c .MaxSizePerMsg ),
456+ maxUncommittedSize : entryPayloadSize (c .MaxUncommittedEntriesSize ),
457+ prs : tracker .MakeProgressTracker (c .MaxInflightMsgs , c .MaxInflightBytes ),
458+ electionTimeout : c .ElectionTick ,
459+ heartbeatTimeout : c .HeartbeatTick ,
460+ logger : c .Logger ,
461+ checkQuorum : c .CheckQuorum ,
462+ preVote : c .PreVote ,
463+ readOnly : newReadOnly (c .ReadOnlyOption ),
464+ disableProposalForwarding : c .DisableProposalForwarding ,
465+ disableConfChangeValidation : c .DisableConfChangeValidation ,
466+ stepDownOnRemoval : c .StepDownOnRemoval ,
467+ disableProposalForwardingCallback : c .DisableProposalForwardingCallback ,
459468 }
460469
461470 cfg , prs , err := confchange .Restore (confchange.Changer {
@@ -1676,6 +1685,11 @@ func stepFollower(r *raft, m pb.Message) error {
16761685 } else if r .disableProposalForwarding {
16771686 r .logger .Infof ("%x not forwarding to leader %x at term %d; dropping proposal" , r .id , r .lead , r .Term )
16781687 return ErrProposalDropped
1688+ } else if r .disableProposalForwardingCallback != nil && r .disableProposalForwardingCallback (m ) {
1689+ r .logger .Infof ("%x not forwarding to leader %x at term %d" +
1690+ " because disableProposalForwardingCallback() returned true for the message; dropping proposal" ,
1691+ r .id , r .lead , r .Term )
1692+ return ErrProposalDropped
16791693 }
16801694 m .To = r .lead
16811695 r .send (m )
0 commit comments