Skip to content

Commit

Permalink
Refactoring the 'suspicion protocol' to a generic module for timebase…
Browse files Browse the repository at this point in the history
…d state transitions.
  • Loading branch information
Sever Banesiu authored and Nils Dijk committed Mar 4, 2016
1 parent 6affd85 commit 99b1219
Show file tree
Hide file tree
Showing 7 changed files with 368 additions and 308 deletions.
2 changes: 1 addition & 1 deletion swim/gossip_test.go
Expand Up @@ -118,7 +118,7 @@ func (s *GossipTestSuite) TestSuspicionStarted() {

s.g.ProtocolPeriod()

s.NotNil(s.node.suspicion.Timer("127.0.0.1:3010"), "expected suspicion timer to be set")
s.NotNil(s.node.stateTransitions.timer("127.0.0.1:3010"), "expected state timer to be set")
}

func TestGossipTestSuite(t *testing.T) {
Expand Down
38 changes: 20 additions & 18 deletions swim/node.go
Expand Up @@ -40,7 +40,7 @@ var (

// Options is a configuration struct passed the NewNode constructor.
type Options struct {
SuspicionTimeout time.Duration
StateTimeouts StateTimeouts
MinProtocolPeriod time.Duration

JoinTimeout, PingTimeout, PingRequestTimeout time.Duration
Expand All @@ -55,7 +55,10 @@ type Options struct {

func defaultOptions() *Options {
opts := &Options{
SuspicionTimeout: 5000 * time.Millisecond,
StateTimeouts: StateTimeouts{
Suspect: 5000 * time.Millisecond,
},

MinProtocolPeriod: 200 * time.Millisecond,

JoinTimeout: 1000 * time.Millisecond,
Expand All @@ -80,8 +83,7 @@ func mergeDefaultOptions(opts *Options) *Options {
return def
}

opts.SuspicionTimeout = util.SelectDuration(opts.SuspicionTimeout,
def.SuspicionTimeout)
opts.StateTimeouts = mergeStateTimeouts(opts.StateTimeouts, def.StateTimeouts)

opts.MinProtocolPeriod = util.SelectDuration(opts.MinProtocolPeriod,
def.MinProtocolPeriod)
Expand Down Expand Up @@ -130,13 +132,13 @@ type Node struct {
sync.RWMutex
}

channel shared.SubChannel
memberlist *memberlist
memberiter memberIter
disseminator *disseminator
suspicion *suspicion
gossip *gossip
rollup *updateRollup
channel shared.SubChannel
memberlist *memberlist
memberiter memberIter
disseminator *disseminator
stateTransitions *stateTransitions
gossip *gossip
rollup *updateRollup

joinTimeout, pingTimeout, pingRequestTimeout time.Duration

Expand Down Expand Up @@ -182,7 +184,7 @@ func NewNode(app, address string, channel shared.SubChannel, opts *Options) *Nod

node.memberlist = newMemberlist(node)
node.memberiter = newMemberlistIter(node.memberlist)
node.suspicion = newSuspicion(node, opts.SuspicionTimeout)
node.stateTransitions = newStateTransitions(node, opts.StateTimeouts)
node.gossip = newGossip(node, opts.MinProtocolPeriod)
node.disseminator = newDisseminator(node)
node.rollup = newUpdateRollup(node, opts.RollupFlushInterval,
Expand Down Expand Up @@ -239,7 +241,7 @@ func (n *Node) RegisterListener(l EventListener) {
// Start starts the SWIM protocol and all sub-protocols.
func (n *Node) Start() {
n.gossip.Start()
n.suspicion.Reenable()
n.stateTransitions.Enable()

n.state.Lock()
n.state.stopped = false
Expand All @@ -249,7 +251,7 @@ func (n *Node) Start() {
// Stop stops the SWIM protocol and all sub-protocols.
func (n *Node) Stop() {
n.gossip.Stop()
n.suspicion.Disable()
n.stateTransitions.Disable()

n.state.Lock()
n.state.stopped = true
Expand Down Expand Up @@ -396,18 +398,18 @@ func (n *Node) handleChanges(changes []Change) {

switch change.Status {
case Alive:
n.suspicion.Stop(change)
n.stateTransitions.Cancel(change)
n.disseminator.AdjustMaxPropagations()

case Faulty:
n.suspicion.Stop(change)
n.stateTransitions.Cancel(change)

case Suspect:
n.suspicion.Start(change)
n.stateTransitions.ScheduleSuspectToFaulty(change)
n.disseminator.AdjustMaxPropagations()

case Leave:
n.suspicion.Stop(change)
n.stateTransitions.Cancel(change)
n.disseminator.AdjustMaxPropagations()
}
}
Expand Down
6 changes: 3 additions & 3 deletions swim/node_test.go
Expand Up @@ -59,11 +59,11 @@ func (s *NodeTestSuite) TestStartStop() {

s.True(s.testNode.node.gossip.Stopped(), "gossip should be stopped")
s.True(s.testNode.node.Stopped(), "node should be stopped")
s.False(s.testNode.node.suspicion.enabled, "suspicion should not be enabled")
s.False(s.testNode.node.stateTransitions.enabled, "suspicion should not be enabled")

s.testNode.node.Start()

s.True(s.testNode.node.suspicion.enabled, "suspicon should be enabled")
s.True(s.testNode.node.stateTransitions.enabled, "suspicon should be enabled")
s.False(s.testNode.node.Stopped(), "node should not be stopped")
s.False(s.testNode.node.gossip.Stopped(), "gossip should not be stopped")
}
Expand All @@ -77,7 +77,7 @@ func (s *NodeTestSuite) TestStoppedBootstrapOption() {
s.True(s.testNode.node.gossip.Stopped(), "gossip should be stopped")
// TODO: Should these also be stopped?
//s.True(s.testNode.node.Stopped(), "node should be stopped")
//s.False(s.testNode.node.suspicion.enabled, "suspicion should not be enabled")
//s.False(s.testNode.node.stateTransitions.enabled, "suspicion should not be enabled")
}

func TestNodeTestSuite(t *testing.T) {
Expand Down
195 changes: 195 additions & 0 deletions swim/state_transitions.go
@@ -0,0 +1,195 @@
// Copyright (c) 2015 Uber Technologies, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.

package swim

import (
"sync"
"time"

"github.com/uber-common/bark"
"github.com/uber/ringpop-go/logging"
"github.com/uber/ringpop-go/util"
)

// subject is an interface to define the subject (eg. member) to transition state for. This interface allows to pass in both a Member and a Change struct to the schedule function.
type subject interface {
address() string
incarnation() int64
}

type transitionTimer struct {
*time.Timer

// state represents the state the subject was in when the transition was scheduled
state string
}

// stateTransitions handles the timers for state transitions in SWIM
type stateTransitions struct {
sync.Mutex

node *Node

timeouts StateTimeouts
timers map[string]*transitionTimer
enabled bool
logger bark.Logger
}

// StateTimeouts contains the configured timeouts for every state before transitioning to the new state
type StateTimeouts struct {
// Suspect is the timeout it takes a node in suspect mode to transition to faulty
Suspect time.Duration
}

func mergeStateTimeouts(one StateTimeouts, two StateTimeouts) StateTimeouts {
return StateTimeouts{
Suspect: util.SelectDuration(one.Suspect, two.Suspect),
}
}

// newStateTransitions returns a new state transition controller that can be used to schedule state transitions for nodes
func newStateTransitions(n *Node, timeouts StateTimeouts) *stateTransitions {
return &stateTransitions{
node: n,
timeouts: timeouts,
timers: make(map[string]*transitionTimer),
enabled: true,
logger: logging.Logger("stateTransitions").WithField("local", n.Address()),
}
}

// ScheduleSuspectToFaulty starts the suspect timer. After the Suspect timeout the node will be declared faulty
func (s *stateTransitions) ScheduleSuspectToFaulty(subject subject) {
s.Lock()
s.schedule(subject, Suspect, s.timeouts.Suspect, func() {
// transition the subject to faulty
s.node.memberlist.MakeFaulty(subject.address(), subject.incarnation())
})
s.Unlock()
}

func (s *stateTransitions) schedule(subject subject, state string, timeout time.Duration, transition func()) {
if !s.enabled {
s.logger.WithField("member", subject.address()).Warn("cannot schedule a state transition while disabled")
return
}

if s.node.Address() == subject.address() {
s.logger.WithField("member", subject.address()).Warn("cannot schedule a state transition for the local member")
return
}

if timer, ok := s.timers[subject.address()]; ok {
if timer.state == state {
s.logger.WithFields(bark.Fields{
"member": subject.address(),
"state": state,
}).Warn("redundant call to schedule a state transition for memer, ignored")
return
}
// cancel the previously scheduled transition for the subject
timer.Stop()
}

timer := time.AfterFunc(timeout, func() {
s.logger.WithFields(bark.Fields{
"member": subject.address(),
"state": state,
}).Info("executing scheduled transition for member")
// execute the transition
transition()
})

s.timers[subject.address()] = &transitionTimer{
Timer: timer,
state: state,
}

s.logger.WithFields(bark.Fields{
"member": subject.address(),
"state": state,
}).Debug("scheduled state transition for member")
}

// Cancel cancels the scheduled transition for the subject
func (s *stateTransitions) Cancel(subject subject) {
s.Lock()

if timer, ok := s.timers[subject.address()]; ok {
timer.Stop()
delete(s.timers, subject.address())
s.logger.WithFields(bark.Fields{
"member": subject.address(),
"state": timer.state,
}).Debug("stopped scheduled state transition for member")
}

s.Unlock()
}

// Enable enables state transition controller. The transition controller needs to be in enabled state to allow transitions to be scheduled.
func (s *stateTransitions) Enable() {
s.Lock()

if s.enabled {
s.logger.Warn("state transition controller already enabled")
s.Unlock()
return
}

s.enabled = true
s.Unlock()
s.logger.Info("enabled state transition controller")
}

// Disable cancels all scheduled state transitions and disables the state transition controller for further use
func (s *stateTransitions) Disable() {
s.Lock()

if !s.enabled {
s.logger.Warn("state transition controller already disabled")
s.Unlock()
return
}

s.enabled = false

numTimers := len(s.timers)
for address, timer := range s.timers {
timer.Stop()
delete(s.timers, address)
}

s.Unlock()
s.logger.WithField("timersStopped", numTimers).Info("disabled state transition controller")
}

// timer is a testing func to avoid data races
func (s *stateTransitions) timer(address string) *time.Timer {
s.Lock()
t, ok := s.timers[address]
s.Unlock()
if !ok {
return nil
}
return t.Timer
}

0 comments on commit 99b1219

Please sign in to comment.