etcd/raft/rafttest/network.go
Tobias Schottdorf ac6b604bb8 raft/rafttest: introduce datadriven testing
It has often been tedious to test the interactions between multi-member
Raft groups, especially when many steps were required to reach a certain
scenario. Often, this boilerplate was as boring as it is hard to write
and hard to maintain, making it attractive to resort to shortcuts
whenever possible, which in turn tended to undercut how meaningful and
maintainable the tests ended up being - that is, if the tests were even
written, which sometimes they weren't.

This change introduces a datadriven framework specifically for testing
deterministically the interaction between multiple members of a raft group
with the goal of reducing the friction for writing these tests to near
zero.

In the near term, this will be used to add thorough testing for joint
consensus (which is already available today, but wildly undertested),
but just converting an existing test into this framework has shown that
the concise representation and built-in inspection of log messages
highlights unexpected behavior much more readily than the previous unit
tests did (the test in question is `snapshot_succeed_via_app_resp`; the
reader is invited to compare the old and new version of it).

The main building block is `InteractionEnv`, which holds on to the state
of the whole system and exposes various relevant methods for
manipulating it, including but not limited to adding nodes, delivering
and dropping messages, and proposing configuration changes. All of this
is extensible so that in the future I hope to use it to explore the
phenomena discussed in

https://github.com/etcd-io/etcd/issues/7625#issuecomment-488798263

which requires injecting appropriate "crash points" in the Ready
handling loop. Discussions of the "what if X happened in state Y"
can quickly be made concrete by "scripting up an interaction test".

Additionally, this framework is intentionally not kept internal to the
raft package.. Though this is in its infancy, a goal is that it should
be possible for a suite of interaction tests to allow applications to
validate that their Storage implementation behaves accordingly, simply
by running a raft-provided interaction suite against their Storage.
2019-08-12 08:10:29 -07:00

186 lines
3.9 KiB
Go

// Copyright 2015 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package rafttest
import (
"math/rand"
"sync"
"time"
"go.etcd.io/etcd/raft/raftpb"
)
// a network interface
type iface interface {
send(m raftpb.Message)
recv() chan raftpb.Message
disconnect()
connect()
}
// a network
type network interface {
// drop message at given rate (1.0 drops all messages)
drop(from, to uint64, rate float64)
// delay message for (0, d] randomly at given rate (1.0 delay all messages)
// do we need rate here?
delay(from, to uint64, d time.Duration, rate float64)
disconnect(id uint64)
connect(id uint64)
// heal heals the network
heal()
}
type raftNetwork struct {
rand *rand.Rand
mu sync.Mutex
disconnected map[uint64]bool
dropmap map[conn]float64
delaymap map[conn]delay
recvQueues map[uint64]chan raftpb.Message
}
type conn struct {
from, to uint64
}
type delay struct {
d time.Duration
rate float64
}
func newRaftNetwork(nodes ...uint64) *raftNetwork {
pn := &raftNetwork{
rand: rand.New(rand.NewSource(1)),
recvQueues: make(map[uint64]chan raftpb.Message),
dropmap: make(map[conn]float64),
delaymap: make(map[conn]delay),
disconnected: make(map[uint64]bool),
}
for _, n := range nodes {
pn.recvQueues[n] = make(chan raftpb.Message, 1024)
}
return pn
}
func (rn *raftNetwork) nodeNetwork(id uint64) iface {
return &nodeNetwork{id: id, raftNetwork: rn}
}
func (rn *raftNetwork) send(m raftpb.Message) {
rn.mu.Lock()
to := rn.recvQueues[m.To]
if rn.disconnected[m.To] {
to = nil
}
drop := rn.dropmap[conn{m.From, m.To}]
dl := rn.delaymap[conn{m.From, m.To}]
rn.mu.Unlock()
if to == nil {
return
}
if drop != 0 && rn.rand.Float64() < drop {
return
}
// TODO: shall we dl without blocking the send call?
if dl.d != 0 && rn.rand.Float64() < dl.rate {
rd := rn.rand.Int63n(int64(dl.d))
time.Sleep(time.Duration(rd))
}
// use marshal/unmarshal to copy message to avoid data race.
b, err := m.Marshal()
if err != nil {
panic(err)
}
var cm raftpb.Message
err = cm.Unmarshal(b)
if err != nil {
panic(err)
}
select {
case to <- cm:
default:
// drop messages when the receiver queue is full.
}
}
func (rn *raftNetwork) recvFrom(from uint64) chan raftpb.Message {
rn.mu.Lock()
fromc := rn.recvQueues[from]
if rn.disconnected[from] {
fromc = nil
}
rn.mu.Unlock()
return fromc
}
func (rn *raftNetwork) drop(from, to uint64, rate float64) {
rn.mu.Lock()
defer rn.mu.Unlock()
rn.dropmap[conn{from, to}] = rate
}
func (rn *raftNetwork) delay(from, to uint64, d time.Duration, rate float64) {
rn.mu.Lock()
defer rn.mu.Unlock()
rn.delaymap[conn{from, to}] = delay{d, rate}
}
func (rn *raftNetwork) heal() {
rn.mu.Lock()
defer rn.mu.Unlock()
rn.dropmap = make(map[conn]float64)
rn.delaymap = make(map[conn]delay)
}
func (rn *raftNetwork) disconnect(id uint64) {
rn.mu.Lock()
defer rn.mu.Unlock()
rn.disconnected[id] = true
}
func (rn *raftNetwork) connect(id uint64) {
rn.mu.Lock()
defer rn.mu.Unlock()
rn.disconnected[id] = false
}
type nodeNetwork struct {
id uint64
*raftNetwork
}
func (nt *nodeNetwork) connect() {
nt.raftNetwork.connect(nt.id)
}
func (nt *nodeNetwork) disconnect() {
nt.raftNetwork.disconnect(nt.id)
}
func (nt *nodeNetwork) send(m raftpb.Message) {
nt.raftNetwork.send(m)
}
func (nt *nodeNetwork) recv() chan raftpb.Message {
return nt.recvFrom(nt.id)
}