raft: allow use of joint quorums

This change introduces joint quorums by changing the Node and RawNode
API to accept pb.ConfChangeV2 (on top of pb.ConfChange).

pb.ConfChange continues to work as today: it allows carrying out a
single configuration change. A pb.ConfChange proposal gets added to
the Raft log as such and is thus also observed by the app during Ready
handling, and fed back to ApplyConfChange.

ConfChangeV2 allows joint configuration changes but will continue to
carry out configuration changes in "one phase" (i.e. without ever
entering a joint config) when this is possible.
This commit is contained in:
Tobias Schottdorf
2019-07-22 22:30:47 +02:00
parent 88f5561733
commit b67303c6a2
17 changed files with 616 additions and 153 deletions

View File

@@ -46,7 +46,7 @@ type Changer struct {
// (Section 4.3) corresponds to `C_{new,old}`.
//
// [1]: https://github.com/ongardie/dissertation/blob/master/online-trim.pdf
func (c Changer) EnterJoint(ccs ...pb.ConfChangeSingle) (tracker.Config, tracker.ProgressMap, error) {
func (c Changer) EnterJoint(autoLeave bool, ccs ...pb.ConfChangeSingle) (tracker.Config, tracker.ProgressMap, error) {
cfg, prs, err := c.checkAndCopy()
if err != nil {
return c.err(err)
@@ -74,7 +74,7 @@ func (c Changer) EnterJoint(ccs ...pb.ConfChangeSingle) (tracker.Config, tracker
if err := c.apply(&cfg, prs, ccs...); err != nil {
return c.err(err)
}
cfg.AutoLeave = autoLeave
return checkAndReturn(cfg, prs)
}
@@ -120,6 +120,7 @@ func (c Changer) LeaveJoint() (tracker.Config, tracker.ProgressMap, error) {
}
}
*outgoingPtr(&cfg.Voters) = nil
cfg.AutoLeave = false
return checkAndReturn(cfg, prs)
}
@@ -142,7 +143,7 @@ func (c Changer) Simple(ccs ...pb.ConfChangeSingle) (tracker.Config, tracker.Pro
return c.err(err)
}
if n := symdiff(incoming(c.Tracker.Voters), incoming(cfg.Voters)); n > 1 {
return tracker.Config{}, nil, errors.New("more than voter changed without entering joint config")
return tracker.Config{}, nil, errors.New("more than one voter changed without entering joint config")
}
if err := checkInvariants(cfg, prs); err != nil {
return tracker.Config{}, tracker.ProgressMap{}, nil
@@ -327,6 +328,9 @@ func checkInvariants(cfg tracker.Config, prs tracker.ProgressMap) error {
if cfg.LearnersNext != nil {
return fmt.Errorf("LearnersNext must be nil when not joint")
}
if cfg.AutoLeave {
return fmt.Errorf("AutoLeave must be false when not joint")
}
}
return nil

View File

@@ -85,7 +85,11 @@ func TestConfChangeDataDriven(t *testing.T) {
case "simple":
cfg, prs, err = c.Simple(ccs...)
case "enter-joint":
cfg, prs, err = c.EnterJoint(ccs...)
var autoLeave bool
if len(d.CmdArgs) > 0 {
d.ScanArgs(t, "autoleave", &autoLeave)
}
cfg, prs, err = c.EnterJoint(autoLeave, ccs...)
case "leave-joint":
if len(ccs) > 0 {
err = errors.New("this command takes no input")

View File

@@ -15,6 +15,7 @@
package confchange
import (
"fmt"
"math/rand"
"reflect"
"testing"
@@ -36,16 +37,38 @@ func TestConfChangeQuick(t *testing.T) {
const infoCount = 5
runWithJoint := func(c *Changer, ccs []pb.ConfChangeSingle) error {
cfg, prs, err := c.EnterJoint(ccs...)
cfg, prs, err := c.EnterJoint(false /* autoLeave */, ccs...)
if err != nil {
return err
}
// Also do this with autoLeave on, just to check that we'd get the same
// result.
cfg2a, prs2a, err := c.EnterJoint(true /* autoLeave */, ccs...)
if err != nil {
return err
}
cfg2a.AutoLeave = false
if !reflect.DeepEqual(cfg, cfg2a) || !reflect.DeepEqual(prs, prs2a) {
return fmt.Errorf("cfg: %+v\ncfg2a: %+v\nprs: %+v\nprs2a: %+v",
cfg, cfg2a, prs, prs2a)
}
c.Tracker.Config = cfg
c.Tracker.Progress = prs
cfg2b, prs2b, err := c.LeaveJoint()
if err != nil {
return err
}
// Reset back to the main branch with autoLeave=false.
c.Tracker.Config = cfg
c.Tracker.Progress = prs
cfg, prs, err = c.LeaveJoint()
if err != nil {
return err
}
if !reflect.DeepEqual(cfg, cfg2b) || !reflect.DeepEqual(prs, prs2b) {
return fmt.Errorf("cfg: %+v\ncfg2b: %+v\nprs: %+v\nprs2b: %+v",
cfg, cfg2b, prs, prs2b)
}
c.Tracker.Config = cfg
c.Tracker.Progress = prs
return nil

View File

@@ -0,0 +1,29 @@
# Test the autoleave argument to EnterJoint. It defaults to false in the
# datadriven tests. The flag has no associated semantics in this package,
# it is simply passed through.
simple
v1
----
voters=(1)
1: StateProbe match=0 next=1
# Autoleave is reflected in the config.
enter-joint autoleave=true
v2 v3
----
voters=(1 2 3)&&(1) autoleave
1: StateProbe match=0 next=1
2: StateProbe match=0 next=2
3: StateProbe match=0 next=2
# Can't enter-joint twice, even if autoleave changes.
enter-joint autoleave=false
----
config is already joint
leave-joint
----
voters=(1 2 3)
1: StateProbe match=0 next=1
2: StateProbe match=0 next=2
3: StateProbe match=0 next=2

View File

@@ -20,7 +20,7 @@ voters=(1 2) learners=(3)
simple
r1 v5
----
more than voter changed without entering joint config
more than one voter changed without entering joint config
simple
r1 r2
@@ -30,12 +30,12 @@ removed all voters
simple
v3 v4
----
more than voter changed without entering joint config
more than one voter changed without entering joint config
simple
l1 v5
----
more than voter changed without entering joint config
more than one voter changed without entering joint config
simple
l1 l2