mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00

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.
160 lines
3.7 KiB
Go
160 lines
3.7 KiB
Go
// Copyright 2019 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 (
|
|
"fmt"
|
|
"strconv"
|
|
"testing"
|
|
|
|
"github.com/cockroachdb/datadriven"
|
|
)
|
|
|
|
// Handle is the entrypoint for data-driven interaction testing. Commands and
|
|
// parameters are parsed from the supplied TestData. Errors during data parsing
|
|
// are reported via the supplied *testing.T; errors from the raft nodes and the
|
|
// storage engine are reported to the output buffer.
|
|
func (env *InteractionEnv) Handle(t *testing.T, d datadriven.TestData) string {
|
|
env.Output.Reset()
|
|
var err error
|
|
switch d.Cmd {
|
|
case "add-nodes":
|
|
// Example:
|
|
//
|
|
// add-nodes <number-of-nodes-to-add> voters=(1 2 3) learners=(4 5) index=2 content=foo
|
|
err = env.handleAddNodes(t, d)
|
|
case "campaign":
|
|
// Example:
|
|
//
|
|
// campaign <id-of-candidate>
|
|
err = env.handleCampaign(t, d)
|
|
case "compact":
|
|
// Example:
|
|
//
|
|
// compact <id> <new-first-index>
|
|
err = env.handleCompact(t, d)
|
|
case "deliver-msgs":
|
|
// Deliver the messages for a given recipient.
|
|
//
|
|
// Example:
|
|
//
|
|
// deliver-msgs <idx>
|
|
err = env.handleDeliverMsgs(t, d)
|
|
case "process-ready":
|
|
// Example:
|
|
//
|
|
// process-ready 3
|
|
err = env.handleProcessReady(t, d)
|
|
case "log-level":
|
|
// Set the log level. NONE disables all output, including from the test
|
|
// harness (except errors).
|
|
//
|
|
// Example:
|
|
//
|
|
// log-level WARN
|
|
err = env.handleLogLevel(t, d)
|
|
case "raft-log":
|
|
// Print the Raft log.
|
|
//
|
|
// Example:
|
|
//
|
|
// raft-log 3
|
|
err = env.handleRaftLog(t, d)
|
|
case "stabilize":
|
|
// Deliver messages to and run process-ready on the set of IDs until
|
|
// no more work is to be done.
|
|
//
|
|
// Example:
|
|
//
|
|
// stabilize 1 4
|
|
err = env.handleStabilize(t, d)
|
|
case "status":
|
|
// Print Raft status.
|
|
//
|
|
// Example:
|
|
//
|
|
// status 5
|
|
err = env.handleStatus(t, d)
|
|
case "tick-heartbeat":
|
|
// Tick a heartbeat interval.
|
|
//
|
|
// Example:
|
|
//
|
|
// tick-heartbeat 3
|
|
err = env.handleTickHeartbeat(t, d)
|
|
case "propose-conf-change":
|
|
// Propose a configuration change.
|
|
//
|
|
// Example:
|
|
//
|
|
// propose-conf-change transition=explicit
|
|
// v1 v3 l4 r5
|
|
//
|
|
// Example:
|
|
//
|
|
// propose-conf-change v1=true
|
|
// v5
|
|
err = env.handleProposeConfChange(t, d)
|
|
default:
|
|
err = fmt.Errorf("unknown command")
|
|
}
|
|
if err != nil {
|
|
env.Output.WriteString(err.Error())
|
|
}
|
|
// NB: the highest log level suppresses all output, including that of the
|
|
// handlers. This comes in useful during setup which can be chatty.
|
|
// However, errors are always logged.
|
|
if env.Output.Len() == 0 {
|
|
return "ok"
|
|
}
|
|
if env.Output.Lvl == len(lvlNames)-1 {
|
|
if err != nil {
|
|
return err.Error()
|
|
}
|
|
return "ok (quiet)"
|
|
}
|
|
return env.Output.String()
|
|
}
|
|
|
|
func firstAsInt(t *testing.T, d datadriven.TestData) int {
|
|
t.Helper()
|
|
n, err := strconv.Atoi(d.CmdArgs[0].Key)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
return n
|
|
}
|
|
|
|
func firstAsNodeIdx(t *testing.T, d datadriven.TestData) int {
|
|
t.Helper()
|
|
n := firstAsInt(t, d)
|
|
return n - 1
|
|
}
|
|
|
|
func ints(t *testing.T, d datadriven.TestData) []int {
|
|
var ints []int
|
|
for i := 0; i < len(d.CmdArgs); i++ {
|
|
if len(d.CmdArgs[i].Vals) != 0 {
|
|
continue
|
|
}
|
|
n, err := strconv.Atoi(d.CmdArgs[i].Key)
|
|
if err != nil {
|
|
t.Fatal(err)
|
|
}
|
|
ints = append(ints, n)
|
|
}
|
|
return ints
|
|
}
|