Browse Source

test: add end-to-end testing framework (#5435)

Partial fix for #5291. For details, see [README.md](https://github.com/tendermint/tendermint/blob/erik/e2e-tests/test/e2e/README.md) and [RFC-001](https://github.com/tendermint/tendermint/blob/master/docs/rfc/rfc-001-end-to-end-testing.md).

This only includes a single test case under `test/e2e/tests/`, as a proof of concept - additional test cases will be submitted separately. A randomized testnet generator will also be submitted separately, there a currently just a handful of static testnets under `test/e2e/networks/`. This will eventually replace the current P2P tests and run in CI.
pull/5454/head
Erik Grinaker 4 years ago
committed by GitHub
parent
commit
250c3aa92e
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
31 changed files with 2722 additions and 5 deletions
  1. +5
    -0
      .dockerignore
  2. +2
    -0
      .gitignore
  3. +1
    -0
      go.mod
  4. +8
    -5
      privval/file.go
  5. +16
    -0
      test/e2e/Makefile
  6. +78
    -0
      test/e2e/README.md
  7. +217
    -0
      test/e2e/app/app.go
  8. +50
    -0
      test/e2e/app/config.go
  9. +173
    -0
      test/e2e/app/main.go
  10. +155
    -0
      test/e2e/app/snapshots.go
  11. +155
    -0
      test/e2e/app/state.go
  12. +32
    -0
      test/e2e/docker/Dockerfile
  13. +10
    -0
      test/e2e/docker/entrypoint
  14. +6
    -0
      test/e2e/docker/entrypoint-builtin
  15. +97
    -0
      test/e2e/networks/ci.toml
  16. +4
    -0
      test/e2e/networks/simple.toml
  17. +1
    -0
      test/e2e/networks/single.toml
  18. +127
    -0
      test/e2e/pkg/manifest.go
  19. +470
    -0
      test/e2e/pkg/testnet.go
  20. +35
    -0
      test/e2e/runner/cleanup.go
  21. +50
    -0
      test/e2e/runner/exec.go
  22. +106
    -0
      test/e2e/runner/load.go
  23. +184
    -0
      test/e2e/runner/main.go
  24. +75
    -0
      test/e2e/runner/perturb.go
  25. +107
    -0
      test/e2e/runner/rpc.go
  26. +360
    -0
      test/e2e/runner/setup.go
  27. +68
    -0
      test/e2e/runner/start.go
  28. +19
    -0
      test/e2e/runner/test.go
  29. +24
    -0
      test/e2e/runner/wait.go
  30. +30
    -0
      test/e2e/tests/app_test.go
  31. +57
    -0
      test/e2e/tests/e2e_test.go

+ 5
- 0
.dockerignore View File

@ -0,0 +1,5 @@
build
test/e2e/build
test/e2e/networks
test/logs
test/p2p/data

+ 2
- 0
.gitignore View File

@ -10,6 +10,8 @@ remote_dump
.revision
vendor
.vagrant
test/e2e/build
test/e2e/networks/*/
test/p2p/data/
test/logs
coverage.txt


+ 1
- 0
go.mod View File

@ -3,6 +3,7 @@ module github.com/tendermint/tendermint
go 1.14
require (
github.com/BurntSushi/toml v0.3.1
github.com/ChainSafe/go-schnorrkel v0.0.0-20200405005733-88cbf1b4c40d
github.com/Workiva/go-datastructures v1.0.52
github.com/fortytw2/leaktest v1.3.0


+ 8
- 5
privval/file.go View File

@ -152,11 +152,8 @@ type FilePV struct {
LastSignState FilePVLastSignState
}
// GenFilePV generates a new validator with randomly generated private key
// and sets the filePaths, but does not call Save().
func GenFilePV(keyFilePath, stateFilePath string) *FilePV {
privKey := ed25519.GenPrivKey()
// NewFilePV generates a new validator from the given key and paths.
func NewFilePV(privKey crypto.PrivKey, keyFilePath, stateFilePath string) *FilePV {
return &FilePV{
Key: FilePVKey{
Address: privKey.PubKey().Address(),
@ -171,6 +168,12 @@ func GenFilePV(keyFilePath, stateFilePath string) *FilePV {
}
}
// GenFilePV generates a new validator with randomly generated private key
// and sets the filePaths, but does not call Save().
func GenFilePV(keyFilePath, stateFilePath string) *FilePV {
return NewFilePV(ed25519.GenPrivKey(), keyFilePath, stateFilePath)
}
// LoadFilePV loads a FilePV from the filePaths. The FilePV handles double
// signing prevention by persisting data to the stateFilePath. If either file path
// does not exist, the program will exit.


+ 16
- 0
test/e2e/Makefile View File

@ -0,0 +1,16 @@
docker:
docker build --tag tendermint/e2e-node -f docker/Dockerfile ../..
ci: runner
./build/runner -f networks/ci.toml
# We need to build support for database backends into the app in
# order to build a binary with a Tendermint node in it (for built-in
# ABCI testing).
app:
go build -o build/app -tags badgerdb,boltdb,cleveldb,rocksdb ./app
runner:
go build -o build/runner ./runner
.PHONY: app ci docker runner

+ 78
- 0
test/e2e/README.md View File

@ -0,0 +1,78 @@
# End-to-End Tests
Spins up and tests Tendermint networks in Docker Compose based on a testnet manifest. To run the CI testnet:
```sh
make docker
make runner
./build/runner -f networks/ci.toml
```
This creates and runs a testnet named `ci` under `networks/ci/` (determined by the manifest filename).
## Testnet Manifests
Testnets are specified as TOML manifests. For an example see [`networks/ci.toml`](networks/ci.toml), and for documentation see [`pkg/manifest.go`](pkg/manifest.go).
## Test Stages
The test runner has the following stages, which can also be executed explicitly by running `./build/runner -f <manifest> <stage>`:
* `setup`: generates configuration files.
* `start`: starts Docker containers.
* `load`: generates a transaction load against the testnet nodes.
* `perturb`: runs any requested perturbations (e.g. node restarts or network disconnects).
* `wait`: waits for a few blocks to be produced, and for all nodes to catch up to it.
* `test`: runs test cases in `tests/` against all nodes in a running testnet.
* `stop`: stops Docker containers.
* `cleanup`: removes configuration files and Docker containers/networks.
* `logs`: outputs all node logs.
## Tests
Test cases are written as normal Go tests in `tests/`. They use a `testNode()` helper which executes each test as a parallel subtest for each node in the network.
### Running Manual Tests
To run tests manually, set the `E2E_MANIFEST` environment variable to the path of the testnet manifest (e.g. `networks/ci.toml`) and run them as normal, e.g.:
```sh
./build/runner -f networks/ci.toml start
E2E_MANIFEST=networks/ci.toml go test -v ./tests/...
```
Optionally, `E2E_NODE` specifies the name of a single testnet node to test.
These environment variables can also be specified in `tests/e2e_test.go` to run tests from an editor or IDE:
```go
func init() {
// This can be used to manually specify a testnet manifest and/or node to
// run tests against. The testnet must have been started by the runner first.
os.Setenv("E2E_MANIFEST", "networks/ci.toml")
os.Setenv("E2E_NODE", "validator01")
}
```
### Debugging Failures
If a command or test fails, the runner simply exits with an error message and non-zero status code. The testnet is left running with data in the testnet directory, and can be inspected with e.g. `docker ps`, `docker logs`, or `./build/runner -f <manifest> logs` or `tail`. To shut down and remove the testnet, run `./build/runner -f <manifest> cleanup`.
## Enabling IPv6
Docker does not enable IPv6 by default. To do so, enter the following in `daemon.json` (or in the Docker for Mac UI under Preferences → Docker Engine):
```json
{
"ipv6": true,
"fixed-cidr-v6": "2001:db8:1::/64"
}
```

+ 217
- 0
test/e2e/app/app.go View File

@ -0,0 +1,217 @@
package main
import (
"bytes"
"encoding/base64"
"errors"
"fmt"
"os"
"path/filepath"
"github.com/tendermint/tendermint/abci/example/code"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/version"
)
// Application is an ABCI application for use by end-to-end tests. It is a
// simple key/value store for strings, storing data in memory and persisting
// to disk as JSON, taking state sync snapshots if requested.
type Application struct {
abci.BaseApplication
logger log.Logger
state *State
snapshots *SnapshotStore
cfg *Config
restoreSnapshot *abci.Snapshot
restoreChunks [][]byte
}
// NewApplication creates the application.
func NewApplication(cfg *Config) (*Application, error) {
state, err := NewState(filepath.Join(cfg.Dir, "state.json"), cfg.PersistInterval)
if err != nil {
return nil, err
}
snapshots, err := NewSnapshotStore(filepath.Join(cfg.Dir, "snapshots"))
if err != nil {
return nil, err
}
return &Application{
logger: log.NewTMLogger(log.NewSyncWriter(os.Stdout)),
state: state,
snapshots: snapshots,
cfg: cfg,
}, nil
}
// Info implements ABCI.
func (app *Application) Info(req abci.RequestInfo) abci.ResponseInfo {
return abci.ResponseInfo{
Version: version.ABCIVersion,
AppVersion: 1,
LastBlockHeight: int64(app.state.Height),
LastBlockAppHash: app.state.Hash,
}
}
// Info implements ABCI.
func (app *Application) InitChain(req abci.RequestInitChain) abci.ResponseInitChain {
var err error
app.state.initialHeight = uint64(req.InitialHeight)
if len(req.AppStateBytes) > 0 {
err = app.state.Import(0, req.AppStateBytes)
if err != nil {
panic(err)
}
}
resp := abci.ResponseInitChain{
AppHash: app.state.Hash,
}
if resp.Validators, err = app.validatorUpdates(0); err != nil {
panic(err)
}
return resp
}
// CheckTx implements ABCI.
func (app *Application) CheckTx(req abci.RequestCheckTx) abci.ResponseCheckTx {
_, _, err := parseTx(req.Tx)
if err != nil {
return abci.ResponseCheckTx{
Code: code.CodeTypeEncodingError,
Log: err.Error(),
}
}
return abci.ResponseCheckTx{Code: code.CodeTypeOK, GasWanted: 1}
}
// DeliverTx implements ABCI.
func (app *Application) DeliverTx(req abci.RequestDeliverTx) abci.ResponseDeliverTx {
key, value, err := parseTx(req.Tx)
if err != nil {
panic(err) // shouldn't happen since we verified it in CheckTx
}
app.state.Set(key, value)
return abci.ResponseDeliverTx{Code: code.CodeTypeOK}
}
// EndBlock implements ABCI.
func (app *Application) EndBlock(req abci.RequestEndBlock) abci.ResponseEndBlock {
var err error
resp := abci.ResponseEndBlock{}
if resp.ValidatorUpdates, err = app.validatorUpdates(uint64(req.Height)); err != nil {
panic(err)
}
return resp
}
// Commit implements ABCI.
func (app *Application) Commit() abci.ResponseCommit {
height, hash, err := app.state.Commit()
if err != nil {
panic(err)
}
if app.cfg.SnapshotInterval > 0 && height%app.cfg.SnapshotInterval == 0 {
snapshot, err := app.snapshots.Create(app.state)
if err != nil {
panic(err)
}
logger.Info("Created state sync snapshot", "height", snapshot.Height)
}
retainHeight := int64(0)
if app.cfg.RetainBlocks > 0 {
retainHeight = int64(height - app.cfg.RetainBlocks + 1)
}
return abci.ResponseCommit{
Data: hash,
RetainHeight: retainHeight,
}
}
// Query implements ABCI.
func (app *Application) Query(req abci.RequestQuery) abci.ResponseQuery {
return abci.ResponseQuery{
Height: int64(app.state.Height),
Key: req.Data,
Value: []byte(app.state.Get(string(req.Data))),
}
}
// ListSnapshots implements ABCI.
func (app *Application) ListSnapshots(req abci.RequestListSnapshots) abci.ResponseListSnapshots {
snapshots, err := app.snapshots.List()
if err != nil {
panic(err)
}
return abci.ResponseListSnapshots{Snapshots: snapshots}
}
// LoadSnapshotChunk implements ABCI.
func (app *Application) LoadSnapshotChunk(req abci.RequestLoadSnapshotChunk) abci.ResponseLoadSnapshotChunk {
chunk, err := app.snapshots.LoadChunk(req.Height, req.Format, req.Chunk)
if err != nil {
panic(err)
}
return abci.ResponseLoadSnapshotChunk{Chunk: chunk}
}
// OfferSnapshot implements ABCI.
func (app *Application) OfferSnapshot(req abci.RequestOfferSnapshot) abci.ResponseOfferSnapshot {
if app.restoreSnapshot != nil {
panic("A snapshot is already being restored")
}
app.restoreSnapshot = req.Snapshot
app.restoreChunks = [][]byte{}
return abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}
}
// ApplySnapshotChunk implements ABCI.
func (app *Application) ApplySnapshotChunk(req abci.RequestApplySnapshotChunk) abci.ResponseApplySnapshotChunk {
if app.restoreSnapshot == nil {
panic("No restore in progress")
}
app.restoreChunks = append(app.restoreChunks, req.Chunk)
if len(app.restoreChunks) == int(app.restoreSnapshot.Chunks) {
bz := []byte{}
for _, chunk := range app.restoreChunks {
bz = append(bz, chunk...)
}
err := app.state.Import(app.restoreSnapshot.Height, bz)
if err != nil {
panic(err)
}
app.restoreSnapshot = nil
app.restoreChunks = nil
}
return abci.ResponseApplySnapshotChunk{Result: abci.ResponseApplySnapshotChunk_ACCEPT}
}
// validatorUpdates generates a validator set update.
func (app *Application) validatorUpdates(height uint64) (abci.ValidatorUpdates, error) {
updates := app.cfg.ValidatorUpdates[fmt.Sprintf("%v", height)]
if len(updates) == 0 {
return nil, nil
}
valUpdates := abci.ValidatorUpdates{}
for keyString, power := range updates {
keyBytes, err := base64.StdEncoding.DecodeString(keyString)
if err != nil {
return nil, fmt.Errorf("invalid base64 pubkey value %q: %w", keyString, err)
}
valUpdates = append(valUpdates, abci.Ed25519ValidatorUpdate(keyBytes, int64(power)))
}
return valUpdates, nil
}
// parseTx parses a tx in 'key=value' format into a key and value.
func parseTx(tx []byte) (string, string, error) {
parts := bytes.Split(tx, []byte("="))
if len(parts) != 2 {
return "", "", fmt.Errorf("invalid tx format: %q", string(tx))
}
if len(parts[0]) == 0 {
return "", "", errors.New("key cannot be empty")
}
return string(parts[0]), string(parts[1]), nil
}

+ 50
- 0
test/e2e/app/config.go View File

@ -0,0 +1,50 @@
package main
import (
"errors"
"fmt"
"github.com/BurntSushi/toml"
)
// Config is the application configuration.
type Config struct {
ChainID string `toml:"chain_id"`
Listen string
Protocol string
Dir string
PersistInterval uint64 `toml:"persist_interval"`
SnapshotInterval uint64 `toml:"snapshot_interval"`
RetainBlocks uint64 `toml:"retain_blocks"`
ValidatorUpdates map[string]map[string]uint8 `toml:"validator_update"`
PrivValServer string `toml:"privval_server"`
PrivValKey string `toml:"privval_key"`
PrivValState string `toml:"privval_state"`
}
// LoadConfig loads the configuration from disk.
func LoadConfig(file string) (*Config, error) {
cfg := &Config{
Listen: "unix:///var/run/app.sock",
Protocol: "socket",
PersistInterval: 1,
}
_, err := toml.DecodeFile(file, &cfg)
if err != nil {
return nil, fmt.Errorf("failed to load config from %q: %w", file, err)
}
return cfg, cfg.Validate()
}
// Validate validates the configuration. We don't do exhaustive config
// validation here, instead relying on Testnet.Validate() to handle it.
func (cfg Config) Validate() error {
switch {
case cfg.ChainID == "":
return errors.New("chain_id parameter is required")
case cfg.Listen == "" && cfg.Protocol != "builtin":
return errors.New("listen parameter is required")
default:
return nil
}
}

+ 173
- 0
test/e2e/app/main.go View File

@ -0,0 +1,173 @@
package main
import (
"errors"
"fmt"
"os"
"path/filepath"
"time"
"github.com/spf13/viper"
"github.com/tendermint/tendermint/abci/server"
"github.com/tendermint/tendermint/config"
tmflags "github.com/tendermint/tendermint/libs/cli/flags"
"github.com/tendermint/tendermint/libs/log"
tmnet "github.com/tendermint/tendermint/libs/net"
"github.com/tendermint/tendermint/node"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/privval"
"github.com/tendermint/tendermint/proxy"
)
var logger = log.NewTMLogger(log.NewSyncWriter(os.Stdout))
// main is the binary entrypoint.
func main() {
if len(os.Args) != 2 {
fmt.Printf("Usage: %v <configfile>", os.Args[0])
return
}
configFile := ""
if len(os.Args) == 2 {
configFile = os.Args[1]
}
if err := run(configFile); err != nil {
logger.Error(err.Error())
os.Exit(1)
}
}
// run runs the application - basically like main() with error handling.
func run(configFile string) error {
cfg, err := LoadConfig(configFile)
if err != nil {
return err
}
switch cfg.Protocol {
case "socket", "grpc":
err = startApp(cfg)
case "builtin":
err = startNode(cfg)
default:
err = fmt.Errorf("invalid protocol %q", cfg.Protocol)
}
if err != nil {
return err
}
// Start remote signer
if cfg.PrivValServer != "" {
if err = startSigner(cfg); err != nil {
return err
}
}
// Apparently there's no way to wait for the server, so we just sleep
for {
time.Sleep(1 * time.Hour)
}
}
// startApp starts the application server, listening for connections from Tendermint.
func startApp(cfg *Config) error {
app, err := NewApplication(cfg)
if err != nil {
return err
}
server, err := server.NewServer(cfg.Listen, cfg.Protocol, app)
if err != nil {
return err
}
err = server.Start()
if err != nil {
return err
}
logger.Info(fmt.Sprintf("Server listening on %v (%v protocol)", cfg.Listen, cfg.Protocol))
return nil
}
// startNode starts a Tendermint node running the application directly. It assumes the Tendermint
// configuration is in $TMHOME/config/tendermint.toml.
//
// FIXME There is no way to simply load the configuration from a file, so we need to pull in Viper.
func startNode(cfg *Config) error {
app, err := NewApplication(cfg)
if err != nil {
return err
}
home := os.Getenv("TMHOME")
if home == "" {
return errors.New("TMHOME not set")
}
viper.AddConfigPath(filepath.Join(home, "config"))
viper.SetConfigName("config")
err = viper.ReadInConfig()
if err != nil {
return err
}
tmcfg := config.DefaultConfig()
err = viper.Unmarshal(tmcfg)
if err != nil {
return err
}
tmcfg.SetRoot(home)
if err = tmcfg.ValidateBasic(); err != nil {
return fmt.Errorf("error in config file: %v", err)
}
if tmcfg.LogFormat == config.LogFormatJSON {
logger = log.NewTMJSONLogger(log.NewSyncWriter(os.Stdout))
}
logger, err = tmflags.ParseLogLevel(tmcfg.LogLevel, logger, config.DefaultLogLevel())
if err != nil {
return err
}
logger = logger.With("module", "main")
nodeKey, err := p2p.LoadOrGenNodeKey(tmcfg.NodeKeyFile())
if err != nil {
return fmt.Errorf("failed to load or gen node key %s: %w", tmcfg.NodeKeyFile(), err)
}
n, err := node.NewNode(tmcfg,
privval.LoadOrGenFilePV(tmcfg.PrivValidatorKeyFile(), tmcfg.PrivValidatorStateFile()),
nodeKey,
proxy.NewLocalClientCreator(app),
node.DefaultGenesisDocProviderFunc(tmcfg),
node.DefaultDBProvider,
node.DefaultMetricsProvider(tmcfg.Instrumentation),
logger,
)
if err != nil {
return err
}
return n.Start()
}
// startSigner starts a signer server connecting to the given endpoint.
func startSigner(cfg *Config) error {
filePV := privval.LoadFilePV(cfg.PrivValKey, cfg.PrivValState)
protocol, address := tmnet.ProtocolAndAddress(cfg.PrivValServer)
var dialFn privval.SocketDialer
switch protocol {
case "tcp":
dialFn = privval.DialTCPFn(address, 3*time.Second, filePV.Key.PrivKey)
case "unix":
dialFn = privval.DialUnixFn(address)
default:
return fmt.Errorf("invalid privval protocol %q", protocol)
}
endpoint := privval.NewSignerDialerEndpoint(logger, dialFn,
privval.SignerDialerEndpointRetryWaitInterval(1*time.Second),
privval.SignerDialerEndpointConnRetries(100))
err := privval.NewSignerServer(endpoint, cfg.ChainID, filePV).Start()
if err != nil {
return err
}
logger.Info(fmt.Sprintf("Remote signer connecting to %v", cfg.PrivValServer))
return nil
}

+ 155
- 0
test/e2e/app/snapshots.go View File

@ -0,0 +1,155 @@
// nolint: gosec
package main
import (
"crypto/sha256"
"encoding/json"
"errors"
"fmt"
"io/ioutil"
"math"
"os"
"path/filepath"
"sync"
abci "github.com/tendermint/tendermint/abci/types"
)
const (
snapshotChunkSize = 1e6
)
// SnapshotStore stores state sync snapshots. Snapshots are stored simply as
// JSON files, and chunks are generated on-the-fly by splitting the JSON data
// into fixed-size chunks.
type SnapshotStore struct {
sync.RWMutex
dir string
metadata []abci.Snapshot
}
// NewSnapshotStore creates a new snapshot store.
func NewSnapshotStore(dir string) (*SnapshotStore, error) {
store := &SnapshotStore{dir: dir}
if err := os.MkdirAll(dir, 0755); err != nil {
return nil, err
}
if err := store.loadMetadata(); err != nil {
return nil, err
}
return store, nil
}
// loadMetadata loads snapshot metadata. Does not take out locks, since it's
// called internally on construction.
func (s *SnapshotStore) loadMetadata() error {
file := filepath.Join(s.dir, "metadata.json")
metadata := []abci.Snapshot{}
bz, err := ioutil.ReadFile(file)
switch {
case errors.Is(err, os.ErrNotExist):
case err != nil:
return fmt.Errorf("failed to load snapshot metadata from %q: %w", file, err)
}
if len(bz) != 0 {
err = json.Unmarshal(bz, &metadata)
if err != nil {
return fmt.Errorf("invalid snapshot data in %q: %w", file, err)
}
}
s.metadata = metadata
return nil
}
// saveMetadata saves snapshot metadata. Does not take out locks, since it's
// called internally from e.g. Create().
func (s *SnapshotStore) saveMetadata() error {
bz, err := json.Marshal(s.metadata)
if err != nil {
return err
}
// save the file to a new file and move it to make saving atomic.
newFile := filepath.Join(s.dir, "metadata.json.new")
file := filepath.Join(s.dir, "metadata.json")
err = ioutil.WriteFile(newFile, bz, 0644) // nolint: gosec
if err != nil {
return err
}
return os.Rename(newFile, file)
}
// Create creates a snapshot of the given application state's key/value pairs.
func (s *SnapshotStore) Create(state *State) (abci.Snapshot, error) {
s.Lock()
defer s.Unlock()
bz, err := state.Export()
if err != nil {
return abci.Snapshot{}, err
}
hash := sha256.Sum256(bz)
snapshot := abci.Snapshot{
Height: state.Height,
Format: 1,
Hash: hash[:],
Chunks: byteChunks(bz),
}
err = ioutil.WriteFile(filepath.Join(s.dir, fmt.Sprintf("%v.json", state.Height)), bz, 0644)
if err != nil {
return abci.Snapshot{}, err
}
s.metadata = append(s.metadata, snapshot)
err = s.saveMetadata()
if err != nil {
return abci.Snapshot{}, err
}
return snapshot, nil
}
// List lists available snapshots.
func (s *SnapshotStore) List() ([]*abci.Snapshot, error) {
s.RLock()
defer s.RUnlock()
snapshots := []*abci.Snapshot{}
for _, snapshot := range s.metadata {
s := snapshot // copy to avoid pointer to range variable
snapshots = append(snapshots, &s)
}
return snapshots, nil
}
// LoadChunk loads a snapshot chunk.
func (s *SnapshotStore) LoadChunk(height uint64, format uint32, chunk uint32) ([]byte, error) {
s.RLock()
defer s.RUnlock()
for _, snapshot := range s.metadata {
if snapshot.Height == height && snapshot.Format == format {
bz, err := ioutil.ReadFile(filepath.Join(s.dir, fmt.Sprintf("%v.json", height)))
if err != nil {
return nil, err
}
return byteChunk(bz, chunk), nil
}
}
return nil, nil
}
// byteChunk returns the chunk at a given index from the full byte slice.
func byteChunk(bz []byte, index uint32) []byte {
start := int(index * snapshotChunkSize)
end := int((index + 1) * snapshotChunkSize)
switch {
case start >= len(bz):
return nil
case end >= len(bz):
return bz[start:]
default:
return bz[start:end]
}
}
// byteChunks calculates the number of chunks in the byte slice.
func byteChunks(bz []byte) uint32 {
return uint32(math.Ceil(float64(len(bz)) / snapshotChunkSize))
}

+ 155
- 0
test/e2e/app/state.go View File

@ -0,0 +1,155 @@
//nolint: gosec
package main
import (
"crypto/sha256"
"encoding/json"
"errors"
"fmt"
"io/ioutil"
"os"
"sort"
"sync"
)
// State is the application state.
type State struct {
sync.RWMutex
Height uint64
Values map[string]string
Hash []byte
// private fields aren't marshalled to disk.
file string
persistInterval uint64
initialHeight uint64
}
// NewState creates a new state.
func NewState(file string, persistInterval uint64) (*State, error) {
state := &State{
Values: make(map[string]string),
file: file,
persistInterval: persistInterval,
}
state.Hash = hashItems(state.Values)
err := state.load()
switch {
case errors.Is(err, os.ErrNotExist):
case err != nil:
return nil, err
}
return state, nil
}
// load loads state from disk. It does not take out a lock, since it is called
// during construction.
func (s *State) load() error {
bz, err := ioutil.ReadFile(s.file)
if err != nil {
return fmt.Errorf("failed to read state from %q: %w", s.file, err)
}
err = json.Unmarshal(bz, s)
if err != nil {
return fmt.Errorf("invalid state data in %q: %w", s.file, err)
}
return nil
}
// save saves the state to disk. It does not take out a lock since it is called
// internally by Commit which does lock.
func (s *State) save() error {
bz, err := json.Marshal(s)
if err != nil {
return fmt.Errorf("failed to marshal state: %w", err)
}
// We write the state to a separate file and move it to the destination, to
// make it atomic.
newFile := fmt.Sprintf("%v.new", s.file)
err = ioutil.WriteFile(newFile, bz, 0644)
if err != nil {
return fmt.Errorf("failed to write state to %q: %w", s.file, err)
}
return os.Rename(newFile, s.file)
}
// Export exports key/value pairs as JSON, used for state sync snapshots.
func (s *State) Export() ([]byte, error) {
s.RLock()
defer s.RUnlock()
return json.Marshal(s.Values)
}
// Import imports key/value pairs from JSON bytes, used for InitChain.AppStateBytes and
// state sync snapshots. It also saves the state once imported.
func (s *State) Import(height uint64, jsonBytes []byte) error {
s.Lock()
defer s.Unlock()
values := map[string]string{}
err := json.Unmarshal(jsonBytes, &values)
if err != nil {
return fmt.Errorf("failed to decode imported JSON data: %w", err)
}
s.Height = height
s.Values = values
s.Hash = hashItems(values)
return s.save()
}
// Get fetches a value. A missing value is returned as an empty string.
func (s *State) Get(key string) string {
s.RLock()
defer s.RUnlock()
return s.Values[key]
}
// Set sets a value. Setting an empty value is equivalent to deleting it.
func (s *State) Set(key, value string) {
s.Lock()
defer s.Unlock()
if value == "" {
delete(s.Values, key)
} else {
s.Values[key] = value
}
}
// Commit commits the current state.
func (s *State) Commit() (uint64, []byte, error) {
s.Lock()
defer s.Unlock()
s.Hash = hashItems(s.Values)
switch {
case s.Height > 0:
s.Height++
case s.initialHeight > 0:
s.Height = s.initialHeight
default:
s.Height = 1
}
if s.persistInterval > 0 && s.Height%s.persistInterval == 0 {
err := s.save()
if err != nil {
return 0, nil, err
}
}
return s.Height, s.Hash, nil
}
// hashItems hashes a set of key/value items.
func hashItems(items map[string]string) []byte {
keys := make([]string, 0, len(items))
for key := range items {
keys = append(keys, key)
}
sort.Strings(keys)
hasher := sha256.New()
for _, key := range keys {
_, _ = hasher.Write([]byte(key))
_, _ = hasher.Write([]byte{0})
_, _ = hasher.Write([]byte(items[key]))
_, _ = hasher.Write([]byte{0})
}
return hasher.Sum(nil)
}

+ 32
- 0
test/e2e/docker/Dockerfile View File

@ -0,0 +1,32 @@
# We need to build in a Linux environment to support C libraries, e.g. RocksDB.
# We use Debian instead of Alpine, so that we can use binary database packages
# instead of spending time compiling them.
FROM golang:1.15
RUN apt-get update -y && apt-get upgrade -y
RUN apt-get install -y libleveldb-dev librocksdb-dev
# Set up build directory /src/tendermint
ENV TENDERMINT_BUILD_OPTIONS badgerdb,boltdb,cleveldb,rocksdb
WORKDIR /src/tendermint
# Fetch dependencies separately (for layer caching)
COPY go.mod go.sum ./
RUN go mod download
# Build Tendermint and install into /usr/bin/tendermint
COPY . .
RUN make build && cp build/tendermint /usr/bin/tendermint
COPY test/e2e/docker/entrypoint* /usr/bin/
RUN cd test/e2e && make app && cp build/app /usr/bin/app
# Set up runtime directory. We don't use a separate runtime image since we need
# e.g. leveldb and rocksdb which are already installed in the build image.
WORKDIR /tendermint
VOLUME /tendermint
ENV TMHOME=/tendermint
EXPOSE 26656 26657 26660
ENTRYPOINT ["/usr/bin/entrypoint"]
CMD ["node"]
STOPSIGNAL SIGTERM

+ 10
- 0
test/e2e/docker/entrypoint View File

@ -0,0 +1,10 @@
#!/usr/bin/env bash
# Forcibly remove any stray UNIX sockets left behind from previous runs
rm -rf /var/run/privval.sock /var/run/app.sock
/usr/bin/app /tendermint/config/app.toml &
sleep 1
/usr/bin/tendermint "$@"

+ 6
- 0
test/e2e/docker/entrypoint-builtin View File

@ -0,0 +1,6 @@
#!/usr/bin/env bash
# Forcibly remove any stray UNIX sockets left behind from previous runs
rm -rf /var/run/privval.sock /var/run/app.sock
/usr/bin/app /tendermint/config/app.toml

+ 97
- 0
test/e2e/networks/ci.toml View File

@ -0,0 +1,97 @@
# This testnet is (will be) run by CI, and attempts to cover a broad range of
# functionality with a single network.
initial_height = 1000
initial_state = { initial01 = "a", initial02 = "b", initial03 = "c" }
[validators]
validator01 = 100
[validator_update.0]
validator01 = 10
validator02 = 20
validator03 = 30
validator04 = 40
[validator_update.1010]
validator05 = 50
# validator03 gets killed and validator05 has lots of perturbations, so weight them low.
[validator_update.1020]
validator01 = 100
validator02 = 100
validator03 = 50
validator04 = 100
validator05 = 50
[node.seed01]
mode = "seed"
[node.validator01]
seeds = ["seed01"]
snapshot_interval = 5
perturb = ["disconnect"]
[node.validator02]
seeds = ["seed01"]
database = "boltdb"
abci_protocol = "tcp"
privval_protocol = "tcp"
persist_interval = 0
# FIXME The WAL gets corrupted when restarted
# https://github.com/tendermint/tendermint/issues/5422
#perturb = ["restart"]
[node.validator03]
seeds = ["seed01"]
database = "badgerdb"
# FIXME Should use grpc, but it has race conditions
# https://github.com/tendermint/tendermint/issues/5439
abci_protocol = "unix"
privval_protocol = "unix"
persist_interval = 3
retain_blocks = 3
# FIXME The WAL gets corrupted when killed
# https://github.com/tendermint/tendermint/issues/5422
#perturb = ["kill"]
[node.validator04]
persistent_peers = ["validator01"]
database = "rocksdb"
abci_protocol = "builtin"
retain_blocks = 1
perturb = ["pause"]
[node.validator05]
start_at = 1005 # Becomes part of the validator set at 1010
seeds = ["seed01"]
database = "cleveldb"
fast_sync = "v0"
# FIXME Should use grpc, but it has race conditions
# https://github.com/tendermint/tendermint/issues/5439
abci_protocol = "tcp"
privval_protocol = "tcp"
# FIXME The WAL gets corrupted when killed
# https://github.com/tendermint/tendermint/issues/5422
#perturb = ["kill", "pause", "disconnect", "restart"]
[node.full01]
start_at = 1010
mode = "full"
# FIXME Should use v1, but it won't catch up since some nodes don't have all blocks
# https://github.com/tendermint/tendermint/issues/5444
fast_sync = "v2"
persistent_peers = ["validator01", "validator02", "validator03", "validator04", "validator05"]
# FIXME The WAL gets corrupted when restarted
# https://github.com/tendermint/tendermint/issues/5422
#perturb = ["restart"]
[node.full02]
start_at = 1015
mode = "full"
fast_sync = "v2"
state_sync = true
seeds = ["seed01"]
# FIXME The WAL gets corrupted when restarted
# https://github.com/tendermint/tendermint/issues/5422
#perturb = ["restart"]

+ 4
- 0
test/e2e/networks/simple.toml View File

@ -0,0 +1,4 @@
[node.validator01]
[node.validator02]
[node.validator03]
[node.validator04]

+ 1
- 0
test/e2e/networks/single.toml View File

@ -0,0 +1 @@
[node.validator]

+ 127
- 0
test/e2e/pkg/manifest.go View File

@ -0,0 +1,127 @@
package e2e
import (
"fmt"
"github.com/BurntSushi/toml"
)
// Manifest represents a TOML testnet manifest.
type Manifest struct {
// IPv6 uses IPv6 networking instead of IPv4. Defaults to IPv4.
IPv6 bool `toml:"ipv6"`
// InitialHeight specifies the initial block height, set in genesis. Defaults to 1.
InitialHeight int64 `toml:"initial_height"`
// InitialState is an initial set of key/value pairs for the application,
// set in genesis. Defaults to nothing.
InitialState map[string]string `toml:"initial_state"`
// Validators is the initial validator set in genesis, given as node names
// and power:
//
// validators = { validator01 = 10; validator02 = 20; validator03 = 30 }
//
// Defaults to all nodes that have mode=validator at power 100. Explicitly
// specifying an empty set will start with no validators in genesis, and
// the application must return the validator set in InitChain via the
// setting validator_update.0 (see below).
Validators *map[string]int64
// ValidatorUpdates is a map of heights to validator names and their power,
// and will be returned by the ABCI application. For example, the following
// changes the power of validator01 and validator02 at height 1000:
//
// [validator_update.1000]
// validator01 = 20
// validator02 = 10
//
// Specifying height 0 returns the validator update during InitChain. The
// application returns the validator updates as-is, i.e. removing a
// validator must be done by returning it with power 0, and any validators
// not specified are not changed.
ValidatorUpdates map[string]map[string]int64 `toml:"validator_update"`
// Nodes specifies the network nodes. At least one node must be given.
Nodes map[string]ManifestNode `toml:"node"`
}
// ManifestNode represents a node in a testnet manifest.
type ManifestNode struct {
// Mode specifies the type of node: "validator", "full", or "seed". Defaults to
// "validator". Full nodes do not get a signing key (a dummy key is generated),
// and seed nodes run in seed mode with the PEX reactor enabled.
Mode string
// Seeds is the list of node names to use as P2P seed nodes. Defaults to none.
Seeds []string
// PersistentPeers is a list of node names to maintain persistent P2P
// connections to. If neither seeds nor persistent peers are specified,
// this defaults to all other nodes in the network.
PersistentPeers []string `toml:"persistent_peers"`
// Database specifies the database backend: "goleveldb", "cleveldb",
// "rocksdb", "boltdb", or "badgerdb". Defaults to goleveldb.
Database string
// ABCIProtocol specifies the protocol used to communicate with the ABCI
// application: "unix", "tcp", "grpc", or "builtin". Defaults to unix.
// builtin will build a complete Tendermint node into the application and
// launch it instead of launching a separate Tendermint process.
ABCIProtocol string `toml:"abci_protocol"`
// PrivvalProtocol specifies the protocol used to sign consensus messages:
// "file", "unix", or "tcp". Defaults to "file". For unix and tcp, the ABCI
// application will launch a remote signer client in a separate goroutine.
// Only nodes with mode=validator will actually make use of this.
PrivvalProtocol string `toml:"privval_protocol"`
// StartAt specifies the block height at which the node will be started. The
// runner will wait for the network to reach at least this block height.
StartAt int64 `toml:"start_at"`
// FastSync specifies the fast sync mode: "" (disable), "v0", "v1", or "v2".
// Defaults to disabled.
FastSync string `toml:"fast_sync"`
// StateSync enables state sync. The runner automatically configures trusted
// block hashes and RPC servers. At least one node in the network must have
// SnapshotInterval set to non-zero, and the state syncing node must have
// StartAt set to an appropriate height where a snapshot is available.
StateSync bool `toml:"state_sync"`
// PersistInterval specifies the height interval at which the application
// will persist state to disk. Defaults to 1 (every height), setting this to
// 0 disables state persistence.
PersistInterval *uint64 `toml:"persist_interval"`
// SnapshotInterval specifies the height interval at which the application
// will take state sync snapshots. Defaults to 0 (disabled).
SnapshotInterval uint64 `toml:"snapshot_interval"`
// RetainBlocks specifies the number of recent blocks to retain. Defaults to
// 0, which retains all blocks. Must be greater that PersistInterval and
// SnapshotInterval.
RetainBlocks uint64 `toml:"retain_blocks"`
// Perturb lists perturbations to apply to the node after it has been
// started and synced with the network:
//
// disconnect: temporarily disconnects the node from the network
// kill: kills the node with SIGKILL then restarts it
// pause: temporarily pauses (freezes) the node
// restart: restarts the node, shutting it down with SIGTERM
Perturb []string
}
// LoadManifest loads a testnet manifest from a file.
func LoadManifest(file string) (Manifest, error) {
manifest := Manifest{}
_, err := toml.DecodeFile(file, &manifest)
if err != nil {
return manifest, fmt.Errorf("failed to load testnet manifest %q: %w", file, err)
}
return manifest, nil
}

+ 470
- 0
test/e2e/pkg/testnet.go View File

@ -0,0 +1,470 @@
//nolint: gosec
package e2e
import (
"errors"
"fmt"
"io"
"math/rand"
"net"
"path/filepath"
"sort"
"strconv"
"strings"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
rpchttp "github.com/tendermint/tendermint/rpc/client/http"
)
const (
randomSeed int64 = 2308084734268
proxyPortFirst uint32 = 5701
networkIPv4 = "10.186.73.0/24"
networkIPv6 = "fd80:b10c::/48"
)
type Mode string
type Protocol string
type Perturbation string
const (
ModeValidator Mode = "validator"
ModeFull Mode = "full"
ModeSeed Mode = "seed"
ProtocolBuiltin Protocol = "builtin"
ProtocolFile Protocol = "file"
ProtocolGRPC Protocol = "grpc"
ProtocolTCP Protocol = "tcp"
ProtocolUNIX Protocol = "unix"
PerturbationDisconnect Perturbation = "disconnect"
PerturbationKill Perturbation = "kill"
PerturbationPause Perturbation = "pause"
PerturbationRestart Perturbation = "restart"
)
// Testnet represents a single testnet.
type Testnet struct {
Name string
File string
Dir string
IP *net.IPNet
InitialHeight int64
InitialState map[string]string
Validators map[*Node]int64
ValidatorUpdates map[int64]map[*Node]int64
Nodes []*Node
}
// Node represents a Tendermint node in a testnet.
type Node struct {
Name string
Testnet *Testnet
Mode Mode
Key crypto.PrivKey
IP net.IP
ProxyPort uint32
StartAt int64
FastSync string
StateSync bool
Database string
ABCIProtocol Protocol
PrivvalProtocol Protocol
PersistInterval uint64
SnapshotInterval uint64
RetainBlocks uint64
Seeds []*Node
PersistentPeers []*Node
Perturbations []Perturbation
}
// LoadTestnet loads a testnet from a manifest file, using the filename to
// determine the testnet name and directory (from the basename of the file).
// The testnet generation must be deterministic, since it is generated
// separately by the runner and the test cases. For this reason, testnets use a
// random seed to generate e.g. keys.
func LoadTestnet(file string) (*Testnet, error) {
manifest, err := LoadManifest(file)
if err != nil {
return nil, err
}
dir := strings.TrimSuffix(file, filepath.Ext(file))
// Set up resource generators. These must be deterministic.
netAddress := networkIPv4
if manifest.IPv6 {
netAddress = networkIPv6
}
_, ipNet, err := net.ParseCIDR(netAddress)
if err != nil {
return nil, fmt.Errorf("invalid IP network address %q: %w", netAddress, err)
}
ipGen := newIPGenerator(ipNet)
keyGen := newKeyGenerator(randomSeed)
proxyPortGen := newPortGenerator(proxyPortFirst)
testnet := &Testnet{
Name: filepath.Base(dir),
File: file,
Dir: dir,
IP: ipGen.Network(),
InitialHeight: 1,
InitialState: manifest.InitialState,
Validators: map[*Node]int64{},
ValidatorUpdates: map[int64]map[*Node]int64{},
Nodes: []*Node{},
}
if manifest.InitialHeight > 0 {
testnet.InitialHeight = manifest.InitialHeight
}
// Set up nodes, in alphabetical order (IPs and ports get same order).
nodeNames := []string{}
for name := range manifest.Nodes {
nodeNames = append(nodeNames, name)
}
sort.Strings(nodeNames)
for _, name := range nodeNames {
nodeManifest := manifest.Nodes[name]
node := &Node{
Name: name,
Testnet: testnet,
Key: keyGen.Generate(),
IP: ipGen.Next(),
ProxyPort: proxyPortGen.Next(),
Mode: ModeValidator,
Database: "goleveldb",
ABCIProtocol: ProtocolUNIX,
PrivvalProtocol: ProtocolFile,
StartAt: nodeManifest.StartAt,
FastSync: nodeManifest.FastSync,
StateSync: nodeManifest.StateSync,
PersistInterval: 1,
SnapshotInterval: nodeManifest.SnapshotInterval,
RetainBlocks: nodeManifest.RetainBlocks,
Perturbations: []Perturbation{},
}
if nodeManifest.Mode != "" {
node.Mode = Mode(nodeManifest.Mode)
}
if nodeManifest.Database != "" {
node.Database = nodeManifest.Database
}
if nodeManifest.ABCIProtocol != "" {
node.ABCIProtocol = Protocol(nodeManifest.ABCIProtocol)
}
if nodeManifest.PrivvalProtocol != "" {
node.PrivvalProtocol = Protocol(nodeManifest.PrivvalProtocol)
}
if nodeManifest.PersistInterval != nil {
node.PersistInterval = *nodeManifest.PersistInterval
}
for _, p := range nodeManifest.Perturb {
node.Perturbations = append(node.Perturbations, Perturbation(p))
}
testnet.Nodes = append(testnet.Nodes, node)
}
// We do a second pass to set up seeds and persistent peers, which allows graph cycles.
for _, node := range testnet.Nodes {
nodeManifest, ok := manifest.Nodes[node.Name]
if !ok {
return nil, fmt.Errorf("failed to look up manifest for node %q", node.Name)
}
for _, seedName := range nodeManifest.Seeds {
seed := testnet.LookupNode(seedName)
if seed == nil {
return nil, fmt.Errorf("unknown seed %q for node %q", seedName, node.Name)
}
node.Seeds = append(node.Seeds, seed)
}
for _, peerName := range nodeManifest.PersistentPeers {
peer := testnet.LookupNode(peerName)
if peer == nil {
return nil, fmt.Errorf("unknown persistent peer %q for node %q", peerName, node.Name)
}
node.PersistentPeers = append(node.PersistentPeers, peer)
}
// If there are no seeds or persistent peers specified, default to persistent
// connections to all other nodes.
if len(node.PersistentPeers) == 0 && len(node.Seeds) == 0 {
for _, peer := range testnet.Nodes {
if peer.Name == node.Name {
continue
}
node.PersistentPeers = append(node.PersistentPeers, peer)
}
}
}
// Set up genesis validators. If not specified explicitly, use all validator nodes.
if manifest.Validators != nil {
for validatorName, power := range *manifest.Validators {
validator := testnet.LookupNode(validatorName)
if validator == nil {
return nil, fmt.Errorf("unknown validator %q", validatorName)
}
testnet.Validators[validator] = power
}
} else {
for _, node := range testnet.Nodes {
if node.Mode == ModeValidator {
testnet.Validators[node] = 100
}
}
}
// Set up validator updates.
for heightStr, validators := range manifest.ValidatorUpdates {
height, err := strconv.Atoi(heightStr)
if err != nil {
return nil, fmt.Errorf("invalid validator update height %q: %w", height, err)
}
valUpdate := map[*Node]int64{}
for name, power := range validators {
node := testnet.LookupNode(name)
if node == nil {
return nil, fmt.Errorf("unknown validator %q for update at height %v", name, height)
}
valUpdate[node] = power
}
testnet.ValidatorUpdates[int64(height)] = valUpdate
}
return testnet, testnet.Validate()
}
// Validate validates a testnet.
func (t Testnet) Validate() error {
if t.Name == "" {
return errors.New("network has no name")
}
if t.IP == nil {
return errors.New("network has no IP")
}
if len(t.Nodes) == 0 {
return errors.New("network has no nodes")
}
for _, node := range t.Nodes {
if err := node.Validate(t); err != nil {
return fmt.Errorf("invalid node %q: %w", node.Name, err)
}
}
return nil
}
// Validate validates a node.
func (n Node) Validate(testnet Testnet) error {
if n.Name == "" {
return errors.New("node has no name")
}
if n.IP == nil {
return errors.New("node has no IP address")
}
if !testnet.IP.Contains(n.IP) {
return fmt.Errorf("node IP %v is not in testnet network %v", n.IP, testnet.IP)
}
if n.ProxyPort > 0 {
if n.ProxyPort <= 1024 {
return fmt.Errorf("local port %v must be >1024", n.ProxyPort)
}
for _, peer := range testnet.Nodes {
if peer.Name != n.Name && peer.ProxyPort == n.ProxyPort {
return fmt.Errorf("peer %q also has local port %v", peer.Name, n.ProxyPort)
}
}
}
switch n.FastSync {
case "", "v0", "v1", "v2":
default:
return fmt.Errorf("invalid fast sync setting %q", n.FastSync)
}
switch n.Database {
case "goleveldb", "cleveldb", "boltdb", "rocksdb", "badgerdb":
default:
return fmt.Errorf("invalid database setting %q", n.Database)
}
switch n.ABCIProtocol {
case ProtocolBuiltin, ProtocolUNIX, ProtocolTCP, ProtocolGRPC:
default:
return fmt.Errorf("invalid ABCI protocol setting %q", n.ABCIProtocol)
}
switch n.PrivvalProtocol {
case ProtocolFile, ProtocolUNIX, ProtocolTCP:
default:
return fmt.Errorf("invalid privval protocol setting %q", n.PrivvalProtocol)
}
if n.StateSync && n.StartAt == 0 {
return errors.New("state synced nodes cannot start at the initial height")
}
if n.PersistInterval == 0 && n.RetainBlocks > 0 {
return errors.New("persist_interval=0 requires retain_blocks=0")
}
if n.PersistInterval > 1 && n.RetainBlocks > 0 && n.RetainBlocks < n.PersistInterval {
return errors.New("persist_interval must be less than or equal to retain_blocks")
}
if n.SnapshotInterval > 0 && n.RetainBlocks > 0 && n.RetainBlocks < n.SnapshotInterval {
return errors.New("snapshot_interval must be less than er equal to retain_blocks")
}
for _, perturbation := range n.Perturbations {
switch perturbation {
case PerturbationDisconnect, PerturbationKill, PerturbationPause, PerturbationRestart:
default:
return fmt.Errorf("invalid perturbation %q", perturbation)
}
}
return nil
}
// LookupNode looks up a node by name. For now, simply do a linear search.
func (t Testnet) LookupNode(name string) *Node {
for _, node := range t.Nodes {
if node.Name == name {
return node
}
}
return nil
}
// ArchiveNodes returns a list of archive nodes that start at the initial height
// and contain the entire blockchain history. They are used e.g. as light client
// RPC servers.
func (t Testnet) ArchiveNodes() []*Node {
nodes := []*Node{}
for _, node := range t.Nodes {
if node.Mode != ModeSeed && node.StartAt == 0 && node.RetainBlocks == 0 {
nodes = append(nodes, node)
}
}
return nodes
}
// RandomNode returns a random non-seed node.
func (t Testnet) RandomNode() *Node {
for {
node := t.Nodes[rand.Intn(len(t.Nodes))]
if node.Mode != ModeSeed {
return node
}
}
}
// IPv6 returns true if the testnet is an IPv6 network.
func (t Testnet) IPv6() bool {
return t.IP.IP.To4() == nil
}
// Address returns a P2P endpoint address for the node.
func (n Node) AddressP2P(withID bool) string {
ip := n.IP.String()
if n.IP.To4() == nil {
// IPv6 addresses must be wrapped in [] to avoid conflict with : port separator
ip = fmt.Sprintf("[%v]", ip)
}
addr := fmt.Sprintf("%v:26656", ip)
if withID {
addr = fmt.Sprintf("%x@%v", n.Key.PubKey().Address().Bytes(), addr)
}
return addr
}
// Address returns an RPC endpoint address for the node.
func (n Node) AddressRPC() string {
ip := n.IP.String()
if n.IP.To4() == nil {
// IPv6 addresses must be wrapped in [] to avoid conflict with : port separator
ip = fmt.Sprintf("[%v]", ip)
}
return fmt.Sprintf("%v:26657", ip)
}
// Client returns an RPC client for a node.
func (n Node) Client() (*rpchttp.HTTP, error) {
return rpchttp.New(fmt.Sprintf("http://127.0.0.1:%v", n.ProxyPort), "/websocket")
}
// keyGenerator generates pseudorandom Ed25519 keys based on a seed.
type keyGenerator struct {
random *rand.Rand
}
func newKeyGenerator(seed int64) *keyGenerator {
return &keyGenerator{
random: rand.New(rand.NewSource(seed)),
}
}
func (g *keyGenerator) Generate() crypto.PrivKey {
seed := make([]byte, ed25519.SeedSize)
_, err := io.ReadFull(g.random, seed)
if err != nil {
panic(err) // this shouldn't happen
}
return ed25519.GenPrivKeyFromSecret(seed)
}
// portGenerator generates local Docker proxy ports for each node.
type portGenerator struct {
nextPort uint32
}
func newPortGenerator(firstPort uint32) *portGenerator {
return &portGenerator{nextPort: firstPort}
}
func (g *portGenerator) Next() uint32 {
port := g.nextPort
g.nextPort++
if g.nextPort == 0 {
panic("port overflow")
}
return port
}
// ipGenerator generates sequential IP addresses for each node, using a random
// network address.
type ipGenerator struct {
network *net.IPNet
nextIP net.IP
}
func newIPGenerator(network *net.IPNet) *ipGenerator {
nextIP := make([]byte, len(network.IP))
copy(nextIP, network.IP)
gen := &ipGenerator{network: network, nextIP: nextIP}
// Skip network and gateway addresses
gen.Next()
gen.Next()
return gen
}
func (g *ipGenerator) Network() *net.IPNet {
n := &net.IPNet{
IP: make([]byte, len(g.network.IP)),
Mask: make([]byte, len(g.network.Mask)),
}
copy(n.IP, g.network.IP)
copy(n.Mask, g.network.Mask)
return n
}
func (g *ipGenerator) Next() net.IP {
ip := make([]byte, len(g.nextIP))
copy(ip, g.nextIP)
for i := len(g.nextIP) - 1; i >= 0; i-- {
g.nextIP[i]++
if g.nextIP[i] != 0 {
break
}
}
return ip
}

+ 35
- 0
test/e2e/runner/cleanup.go View File

@ -0,0 +1,35 @@
package main
import (
"errors"
"fmt"
"os"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
// Cleanup removes the Docker Compose containers and testnet directory.
func Cleanup(testnet *e2e.Testnet) error {
if testnet.Dir == "" {
return errors.New("no directory set")
}
_, err := os.Stat(testnet.Dir)
if os.IsNotExist(err) {
return nil
} else if err != nil {
return err
}
logger.Info("Removing Docker containers and networks")
err = execCompose(testnet.Dir, "down")
if err != nil {
return err
}
logger.Info(fmt.Sprintf("Removing testnet directory %q", testnet.Dir))
err = os.RemoveAll(testnet.Dir)
if err != nil {
return err
}
return nil
}

+ 50
- 0
test/e2e/runner/exec.go View File

@ -0,0 +1,50 @@
//nolint: gosec
package main
import (
"fmt"
"os"
osexec "os/exec"
"path/filepath"
)
// execute executes a shell command.
func exec(args ...string) error {
cmd := osexec.Command(args[0], args[1:]...)
out, err := cmd.CombinedOutput()
switch err := err.(type) {
case nil:
return nil
case *osexec.ExitError:
return fmt.Errorf("failed to run %q:\n%v", args, string(out))
default:
return err
}
}
// execVerbose executes a shell command while displaying its output.
func execVerbose(args ...string) error {
cmd := osexec.Command(args[0], args[1:]...)
cmd.Stdout = os.Stdout
cmd.Stderr = os.Stderr
return cmd.Run()
}
// execCompose runs a Docker Compose command for a testnet.
func execCompose(dir string, args ...string) error {
return exec(append(
[]string{"docker-compose", "-f", filepath.Join(dir, "docker-compose.yml")},
args...)...)
}
// execComposeVerbose runs a Docker Compose command for a testnet and displays its output.
func execComposeVerbose(dir string, args ...string) error {
return execVerbose(append(
[]string{"docker-compose", "-f", filepath.Join(dir, "docker-compose.yml")},
args...)...)
}
// execDocker runs a Docker command.
func execDocker(args ...string) error {
return exec(append([]string{"docker"}, args...)...)
}

+ 106
- 0
test/e2e/runner/load.go View File

@ -0,0 +1,106 @@
package main
import (
"context"
"crypto/rand"
"errors"
"fmt"
"math"
"time"
rpchttp "github.com/tendermint/tendermint/rpc/client/http"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
"github.com/tendermint/tendermint/types"
)
// Load generates transactions against the network until the given
// context is cancelled.
func Load(ctx context.Context, testnet *e2e.Testnet) error {
concurrency := 50
initialTimeout := 1 * time.Minute
stallTimeout := 15 * time.Second
chTx := make(chan types.Tx)
chSuccess := make(chan types.Tx)
ctx, cancel := context.WithCancel(ctx)
defer cancel()
// Spawn job generator and processors.
logger.Info("Starting transaction load...")
started := time.Now()
go loadGenerate(ctx, chTx)
for w := 0; w < concurrency; w++ {
go loadProcess(ctx, testnet, chTx, chSuccess)
}
// Monitor successful transactions, and abort on stalls.
success := 0
timeout := initialTimeout
for {
select {
case <-chSuccess:
success++
timeout = stallTimeout
case <-time.After(timeout):
return fmt.Errorf("unable to submit transactions for %v", timeout)
case <-ctx.Done():
if success == 0 {
return errors.New("failed to submit any transactions")
}
logger.Info(fmt.Sprintf("Ending transaction load after %v txs (%.1f tx/s)...",
success, float64(success)/time.Since(started).Seconds()))
return nil
}
}
}
// loadGenerate generates jobs until the context is cancelled
func loadGenerate(ctx context.Context, chTx chan<- types.Tx) {
for i := 0; i < math.MaxInt64; i++ {
// We keep generating the same 1000 keys over and over, with different values.
// This gives a reasonable load without putting too much data in the app.
id := i % 1000
bz := make([]byte, 2048) // 4kb hex-encoded
_, err := rand.Read(bz)
if err != nil {
panic(fmt.Sprintf("Failed to read random bytes: %v", err))
}
tx := types.Tx(fmt.Sprintf("load-%X=%x", id, bz))
select {
case chTx <- tx:
time.Sleep(10 * time.Millisecond)
case <-ctx.Done():
close(chTx)
return
}
}
}
// loadProcess processes transactions
func loadProcess(ctx context.Context, testnet *e2e.Testnet, chTx <-chan types.Tx, chSuccess chan<- types.Tx) {
// Each worker gets its own client to each node, which allows for some
// concurrency while still bounding it.
clients := map[string]*rpchttp.HTTP{}
var err error
for tx := range chTx {
node := testnet.RandomNode()
client, ok := clients[node.Name]
if !ok {
client, err = node.Client()
if err != nil {
continue
}
clients[node.Name] = client
}
_, err = client.BroadcastTxCommit(ctx, tx)
if err != nil {
continue
}
chSuccess <- tx
}
}

+ 184
- 0
test/e2e/runner/main.go View File

@ -0,0 +1,184 @@
package main
import (
"context"
"fmt"
"os"
"github.com/spf13/cobra"
"github.com/tendermint/tendermint/libs/log"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
var logger = log.NewTMLogger(log.NewSyncWriter(os.Stdout))
func main() {
NewCLI().Run()
}
// CLI is the Cobra-based command-line interface.
type CLI struct {
root *cobra.Command
testnet *e2e.Testnet
}
// NewCLI sets up the CLI.
func NewCLI() *CLI {
cli := &CLI{}
cli.root = &cobra.Command{
Use: "runner",
Short: "End-to-end test runner",
SilenceUsage: true,
SilenceErrors: true, // we'll output them ourselves in Run()
PersistentPreRunE: func(cmd *cobra.Command, args []string) error {
file, err := cmd.Flags().GetString("file")
if err != nil {
return err
}
testnet, err := e2e.LoadTestnet(file)
if err != nil {
return err
}
cli.testnet = testnet
return nil
},
RunE: func(cmd *cobra.Command, args []string) error {
if err := Cleanup(cli.testnet); err != nil {
return err
}
if err := Setup(cli.testnet); err != nil {
return err
}
chLoadResult := make(chan error)
ctx, loadCancel := context.WithCancel(context.Background())
defer loadCancel()
go func() {
err := Load(ctx, cli.testnet)
if err != nil {
logger.Error(fmt.Sprintf("Transaction load failed: %v", err.Error()))
}
chLoadResult <- err
}()
if err := Start(cli.testnet); err != nil {
return err
}
if err := Perturb(cli.testnet); err != nil {
return err
}
if err := Wait(cli.testnet, 5); err != nil { // wait for network to settle
return err
}
loadCancel()
if err := <-chLoadResult; err != nil {
return err
}
if err := Wait(cli.testnet, 3); err != nil { // wait for last txs to commit
return err
}
if err := Test(cli.testnet); err != nil {
return err
}
if err := Cleanup(cli.testnet); err != nil {
return err
}
return nil
},
}
cli.root.PersistentFlags().StringP("file", "f", "", "Testnet TOML manifest")
_ = cli.root.MarkPersistentFlagRequired("file")
cli.root.AddCommand(&cobra.Command{
Use: "setup",
Short: "Generates the testnet directory and configuration",
RunE: func(cmd *cobra.Command, args []string) error {
return Setup(cli.testnet)
},
})
cli.root.AddCommand(&cobra.Command{
Use: "start",
Short: "Starts the Docker testnet, waiting for nodes to become available",
RunE: func(cmd *cobra.Command, args []string) error {
_, err := os.Stat(cli.testnet.Dir)
if os.IsNotExist(err) {
err = Setup(cli.testnet)
}
if err != nil {
return err
}
return Start(cli.testnet)
},
})
cli.root.AddCommand(&cobra.Command{
Use: "perturb",
Short: "Perturbs the Docker testnet, e.g. by restarting or disconnecting nodes",
RunE: func(cmd *cobra.Command, args []string) error {
return Perturb(cli.testnet)
},
})
cli.root.AddCommand(&cobra.Command{
Use: "wait",
Short: "Waits for a few blocks to be produced and all nodes to catch up",
RunE: func(cmd *cobra.Command, args []string) error {
return Wait(cli.testnet, 5)
},
})
cli.root.AddCommand(&cobra.Command{
Use: "stop",
Short: "Stops the Docker testnet",
RunE: func(cmd *cobra.Command, args []string) error {
logger.Info("Stopping testnet")
return execCompose(cli.testnet.Dir, "down")
},
})
cli.root.AddCommand(&cobra.Command{
Use: "load",
Short: "Generates transaction load until the command is cancelled",
RunE: func(cmd *cobra.Command, args []string) error {
return Load(context.Background(), cli.testnet)
},
})
cli.root.AddCommand(&cobra.Command{
Use: "test",
Short: "Runs test cases against a running testnet",
RunE: func(cmd *cobra.Command, args []string) error {
return Test(cli.testnet)
},
})
cli.root.AddCommand(&cobra.Command{
Use: "cleanup",
Short: "Removes the testnet directory",
RunE: func(cmd *cobra.Command, args []string) error {
return Cleanup(cli.testnet)
},
})
cli.root.AddCommand(&cobra.Command{
Use: "logs",
Short: "Shows the testnet logs",
RunE: func(cmd *cobra.Command, args []string) error {
return execComposeVerbose(cli.testnet.Dir, "logs", "--follow")
},
})
return cli
}
// Run runs the CLI.
func (cli *CLI) Run() {
if err := cli.root.Execute(); err != nil {
logger.Error(err.Error())
os.Exit(1)
}
}

+ 75
- 0
test/e2e/runner/perturb.go View File

@ -0,0 +1,75 @@
package main
import (
"fmt"
"time"
rpctypes "github.com/tendermint/tendermint/rpc/core/types"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
// Perturbs a running testnet.
func Perturb(testnet *e2e.Testnet) error {
for _, node := range testnet.Nodes {
for _, perturbation := range node.Perturbations {
_, err := PerturbNode(node, perturbation)
if err != nil {
return err
}
time.Sleep(3 * time.Second) // give network some time to recover between each
}
}
return nil
}
// PerturbNode perturbs a node with a given perturbation, returning its status
// after recovering.
func PerturbNode(node *e2e.Node, perturbation e2e.Perturbation) (*rpctypes.ResultStatus, error) {
testnet := node.Testnet
switch perturbation {
case e2e.PerturbationDisconnect:
logger.Info(fmt.Sprintf("Disconnecting node %v...", node.Name))
if err := execDocker("network", "disconnect", testnet.Name+"_"+testnet.Name, node.Name); err != nil {
return nil, err
}
time.Sleep(10 * time.Second)
if err := execDocker("network", "connect", testnet.Name+"_"+testnet.Name, node.Name); err != nil {
return nil, err
}
case e2e.PerturbationKill:
logger.Info(fmt.Sprintf("Killing node %v...", node.Name))
if err := execCompose(testnet.Dir, "kill", "-s", "SIGKILL", node.Name); err != nil {
return nil, err
}
if err := execCompose(testnet.Dir, "start", node.Name); err != nil {
return nil, err
}
case e2e.PerturbationPause:
logger.Info(fmt.Sprintf("Pausing node %v...", node.Name))
if err := execCompose(testnet.Dir, "pause", node.Name); err != nil {
return nil, err
}
time.Sleep(10 * time.Second)
if err := execCompose(testnet.Dir, "unpause", node.Name); err != nil {
return nil, err
}
case e2e.PerturbationRestart:
logger.Info(fmt.Sprintf("Restarting node %v...", node.Name))
if err := execCompose(testnet.Dir, "restart", node.Name); err != nil {
return nil, err
}
default:
return nil, fmt.Errorf("unexpected perturbation %q", perturbation)
}
status, err := waitForNode(node, 0, 10*time.Second)
if err != nil {
return nil, err
}
logger.Info(fmt.Sprintf("Node %v recovered at height %v", node.Name, status.SyncInfo.LatestBlockHeight))
return status, nil
}

+ 107
- 0
test/e2e/runner/rpc.go View File

@ -0,0 +1,107 @@
package main
import (
"context"
"errors"
"fmt"
"time"
rpchttp "github.com/tendermint/tendermint/rpc/client/http"
rpctypes "github.com/tendermint/tendermint/rpc/core/types"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
"github.com/tendermint/tendermint/types"
)
// waitForHeight waits for the network to reach a certain height (or above),
// returning the highest height seen. Errors if the network is not making
// progress at all.
func waitForHeight(testnet *e2e.Testnet, height int64) (*types.Block, *types.BlockID, error) {
var (
err error
maxResult *rpctypes.ResultBlock
clients = map[string]*rpchttp.HTTP{}
lastIncrease = time.Now()
)
for {
for _, node := range testnet.Nodes {
if node.Mode == e2e.ModeSeed {
continue
}
client, ok := clients[node.Name]
if !ok {
client, err = node.Client()
if err != nil {
continue
}
clients[node.Name] = client
}
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
defer cancel()
result, err := client.Block(ctx, nil)
if err != nil {
continue
}
if result.Block != nil && (maxResult == nil || result.Block.Height >= maxResult.Block.Height) {
maxResult = result
lastIncrease = time.Now()
}
if maxResult != nil && maxResult.Block.Height >= height {
return maxResult.Block, &maxResult.BlockID, nil
}
}
if len(clients) == 0 {
return nil, nil, errors.New("unable to connect to any network nodes")
}
if time.Since(lastIncrease) >= 10*time.Second {
if maxResult == nil {
return nil, nil, errors.New("chain stalled at unknown height")
}
return nil, nil, fmt.Errorf("chain stalled at height %v", maxResult.Block.Height)
}
time.Sleep(1 * time.Second)
}
}
// waitForNode waits for a node to become available and catch up to the given block height.
func waitForNode(node *e2e.Node, height int64, timeout time.Duration) (*rpctypes.ResultStatus, error) {
client, err := node.Client()
if err != nil {
return nil, err
}
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
for {
status, err := client.Status(ctx)
switch {
case errors.Is(err, context.DeadlineExceeded):
return nil, fmt.Errorf("timed out waiting for %v to reach height %v", node.Name, height)
case errors.Is(err, context.Canceled):
return nil, err
case err == nil && status.SyncInfo.LatestBlockHeight >= height:
return status, nil
}
time.Sleep(200 * time.Millisecond)
}
}
// waitForAllNodes waits for all nodes to become available and catch up to the given block height.
func waitForAllNodes(testnet *e2e.Testnet, height int64, timeout time.Duration) (int64, error) {
lastHeight := int64(0)
for _, node := range testnet.Nodes {
if node.Mode == e2e.ModeSeed {
continue
}
status, err := waitForNode(node, height, 20*time.Second)
if err != nil {
return 0, err
}
if status.SyncInfo.LatestBlockHeight > lastHeight {
lastHeight = status.SyncInfo.LatestBlockHeight
}
}
return lastHeight, nil
}

+ 360
- 0
test/e2e/runner/setup.go View File

@ -0,0 +1,360 @@
// nolint: gosec
package main
import (
"bytes"
"encoding/base64"
"encoding/json"
"errors"
"fmt"
"io/ioutil"
"os"
"path/filepath"
"regexp"
"sort"
"strings"
"text/template"
"time"
"github.com/BurntSushi/toml"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/privval"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
"github.com/tendermint/tendermint/types"
)
const (
AppAddressTCP = "tcp://127.0.0.1:30000"
AppAddressUNIX = "unix:///var/run/app.sock"
PrivvalAddressTCP = "tcp://0.0.0.0:27559"
PrivvalAddressUNIX = "unix:///var/run/privval.sock"
PrivvalKeyFile = "config/priv_validator_key.json"
PrivvalStateFile = "data/priv_validator_state.json"
PrivvalDummyKeyFile = "config/dummy_validator_key.json"
PrivvalDummyStateFile = "data/dummy_validator_state.json"
)
// Setup sets up the testnet configuration.
func Setup(testnet *e2e.Testnet) error {
logger.Info(fmt.Sprintf("Generating testnet files in %q", testnet.Dir))
err := os.MkdirAll(testnet.Dir, os.ModePerm)
if err != nil {
return err
}
compose, err := MakeDockerCompose(testnet)
if err != nil {
return err
}
err = ioutil.WriteFile(filepath.Join(testnet.Dir, "docker-compose.yml"), compose, 0644)
if err != nil {
return err
}
genesis, err := MakeGenesis(testnet)
if err != nil {
return err
}
for _, node := range testnet.Nodes {
nodeDir := filepath.Join(testnet.Dir, node.Name)
dirs := []string{
filepath.Join(nodeDir, "config"),
filepath.Join(nodeDir, "data"),
filepath.Join(nodeDir, "data", "app"),
}
for _, dir := range dirs {
err := os.MkdirAll(dir, 0755)
if err != nil {
return err
}
}
err = genesis.SaveAs(filepath.Join(nodeDir, "config", "genesis.json"))
if err != nil {
return err
}
cfg, err := MakeConfig(node)
if err != nil {
return err
}
config.WriteConfigFile(filepath.Join(nodeDir, "config", "config.toml"), cfg) // panics
appCfg, err := MakeAppConfig(node)
if err != nil {
return err
}
err = ioutil.WriteFile(filepath.Join(nodeDir, "config", "app.toml"), appCfg, 0644)
if err != nil {
return err
}
err = (&p2p.NodeKey{PrivKey: node.Key}).SaveAs(filepath.Join(nodeDir, "config", "node_key.json"))
if err != nil {
return err
}
(privval.NewFilePV(node.Key,
filepath.Join(nodeDir, PrivvalKeyFile),
filepath.Join(nodeDir, PrivvalStateFile),
)).Save()
// Set up a dummy validator. Tendermint requires a file PV even when not used, so we
// give it a dummy such that it will fail if it actually tries to use it.
(privval.NewFilePV(ed25519.GenPrivKey(),
filepath.Join(nodeDir, PrivvalDummyKeyFile),
filepath.Join(nodeDir, PrivvalDummyStateFile),
)).Save()
}
return nil
}
// MakeDockerCompose generates a Docker Compose config for a testnet.
func MakeDockerCompose(testnet *e2e.Testnet) ([]byte, error) {
// Must use version 2 Docker Compose format, to support IPv6.
tmpl, err := template.New("docker-compose").Parse(`version: '2.4'
networks:
{{ .Name }}:
driver: bridge
{{- if .IPv6 }}
enable_ipv6: true
{{- end }}
ipam:
driver: default
config:
- subnet: {{ .IP }}
services:
{{- range .Nodes }}
{{ .Name }}:
container_name: {{ .Name }}
image: tendermint/e2e-node
{{- if eq .ABCIProtocol "builtin" }}
entrypoint: /usr/bin/entrypoint-builtin
{{- end }}
init: true
ports:
- 26656
- {{ if .ProxyPort }}{{ .ProxyPort }}:{{ end }}26657
volumes:
- ./{{ .Name }}:/tendermint
networks:
{{ $.Name }}:
ipv{{ if $.IPv6 }}6{{ else }}4{{ end}}_address: {{ .IP }}
{{end}}`)
if err != nil {
return nil, err
}
var buf bytes.Buffer
err = tmpl.Execute(&buf, testnet)
if err != nil {
return nil, err
}
return buf.Bytes(), nil
}
// MakeGenesis generates a genesis document.
func MakeGenesis(testnet *e2e.Testnet) (types.GenesisDoc, error) {
genesis := types.GenesisDoc{
GenesisTime: time.Now(),
ChainID: testnet.Name,
ConsensusParams: types.DefaultConsensusParams(),
InitialHeight: testnet.InitialHeight,
}
for validator, power := range testnet.Validators {
genesis.Validators = append(genesis.Validators, types.GenesisValidator{
Name: validator.Name,
Address: validator.Key.PubKey().Address(),
PubKey: validator.Key.PubKey(),
Power: power,
})
}
// The validator set will be sorted internally by Tendermint ranked by power,
// but we sort it here as well so that all genesis files are identical.
sort.Slice(genesis.Validators, func(i, j int) bool {
return strings.Compare(genesis.Validators[i].Name, genesis.Validators[j].Name) == -1
})
if len(testnet.InitialState) > 0 {
appState, err := json.Marshal(testnet.InitialState)
if err != nil {
return genesis, err
}
genesis.AppState = appState
}
return genesis, genesis.ValidateAndComplete()
}
// MakeConfig generates a Tendermint config for a node.
func MakeConfig(node *e2e.Node) (*config.Config, error) {
cfg := config.DefaultConfig()
cfg.Moniker = node.Name
cfg.ProxyApp = AppAddressTCP
cfg.RPC.ListenAddress = "tcp://0.0.0.0:26657"
cfg.P2P.ExternalAddress = fmt.Sprintf("tcp://%v", node.AddressP2P(false))
cfg.P2P.AddrBookStrict = false
cfg.DBBackend = node.Database
cfg.StateSync.DiscoveryTime = 5 * time.Second
switch node.ABCIProtocol {
case e2e.ProtocolUNIX:
cfg.ProxyApp = AppAddressUNIX
case e2e.ProtocolTCP:
cfg.ProxyApp = AppAddressTCP
case e2e.ProtocolGRPC:
cfg.ProxyApp = AppAddressTCP
cfg.ABCI = "grpc"
case e2e.ProtocolBuiltin:
cfg.ProxyApp = ""
cfg.ABCI = ""
default:
return nil, fmt.Errorf("unexpected ABCI protocol setting %q", node.ABCIProtocol)
}
// Tendermint errors if it does not have a privval key set up, regardless of whether
// it's actually needed (e.g. for remote KMS or non-validators). We set up a dummy
// key here by default, and use the real key for actual validators that should use
// the file privval.
cfg.PrivValidatorListenAddr = ""
cfg.PrivValidatorKey = PrivvalDummyKeyFile
cfg.PrivValidatorState = PrivvalDummyStateFile
switch node.Mode {
case e2e.ModeValidator:
switch node.PrivvalProtocol {
case e2e.ProtocolFile:
cfg.PrivValidatorKey = PrivvalKeyFile
cfg.PrivValidatorState = PrivvalStateFile
case e2e.ProtocolUNIX:
cfg.PrivValidatorListenAddr = PrivvalAddressUNIX
case e2e.ProtocolTCP:
cfg.PrivValidatorListenAddr = PrivvalAddressTCP
default:
return nil, fmt.Errorf("invalid privval protocol setting %q", node.PrivvalProtocol)
}
case e2e.ModeSeed:
cfg.P2P.SeedMode = true
cfg.P2P.PexReactor = true
case e2e.ModeFull:
// Don't need to do anything, since we're using a dummy privval key by default.
default:
return nil, fmt.Errorf("unexpected mode %q", node.Mode)
}
if node.FastSync == "" {
cfg.FastSyncMode = false
} else {
cfg.FastSync.Version = node.FastSync
}
if node.StateSync {
cfg.StateSync.Enable = true
cfg.StateSync.RPCServers = []string{}
for _, peer := range node.Testnet.ArchiveNodes() {
if peer.Name == node.Name {
continue
}
cfg.StateSync.RPCServers = append(cfg.StateSync.RPCServers, peer.AddressRPC())
}
if len(cfg.StateSync.RPCServers) < 2 {
return nil, errors.New("unable to find 2 suitable state sync RPC servers")
}
}
cfg.P2P.Seeds = ""
for _, seed := range node.Seeds {
if len(cfg.P2P.Seeds) > 0 {
cfg.P2P.Seeds += ","
}
cfg.P2P.Seeds += seed.AddressP2P(true)
}
cfg.P2P.PersistentPeers = ""
for _, peer := range node.PersistentPeers {
if len(cfg.P2P.PersistentPeers) > 0 {
cfg.P2P.PersistentPeers += ","
}
cfg.P2P.PersistentPeers += peer.AddressP2P(true)
}
return cfg, nil
}
// MakeAppConfig generates an ABCI application config for a node.
func MakeAppConfig(node *e2e.Node) ([]byte, error) {
cfg := map[string]interface{}{
"chain_id": node.Testnet.Name,
"dir": "data/app",
"listen": AppAddressUNIX,
"protocol": "socket",
"persist_interval": node.PersistInterval,
"snapshot_interval": node.SnapshotInterval,
"retain_blocks": node.RetainBlocks,
}
switch node.ABCIProtocol {
case e2e.ProtocolUNIX:
cfg["listen"] = AppAddressUNIX
case e2e.ProtocolTCP:
cfg["listen"] = AppAddressTCP
case e2e.ProtocolGRPC:
cfg["listen"] = AppAddressTCP
cfg["protocol"] = "grpc"
case e2e.ProtocolBuiltin:
delete(cfg, "listen")
cfg["protocol"] = "builtin"
default:
return nil, fmt.Errorf("unexpected ABCI protocol setting %q", node.ABCIProtocol)
}
switch node.PrivvalProtocol {
case e2e.ProtocolFile:
case e2e.ProtocolTCP:
cfg["privval_server"] = PrivvalAddressTCP
cfg["privval_key"] = PrivvalKeyFile
cfg["privval_state"] = PrivvalStateFile
case e2e.ProtocolUNIX:
cfg["privval_server"] = PrivvalAddressUNIX
cfg["privval_key"] = PrivvalKeyFile
cfg["privval_state"] = PrivvalStateFile
default:
return nil, fmt.Errorf("unexpected privval protocol setting %q", node.PrivvalProtocol)
}
if len(node.Testnet.ValidatorUpdates) > 0 {
validatorUpdates := map[string]map[string]int64{}
for height, validators := range node.Testnet.ValidatorUpdates {
updateVals := map[string]int64{}
for node, power := range validators {
updateVals[base64.StdEncoding.EncodeToString(node.Key.PubKey().Bytes())] = power
}
validatorUpdates[fmt.Sprintf("%v", height)] = updateVals
}
cfg["validator_update"] = validatorUpdates
}
var buf bytes.Buffer
err := toml.NewEncoder(&buf).Encode(cfg)
if err != nil {
return nil, fmt.Errorf("failed to generate app config: %w", err)
}
return buf.Bytes(), nil
}
// UpdateConfigStateSync updates the state sync config for a node.
func UpdateConfigStateSync(node *e2e.Node, height int64, hash []byte) error {
cfgPath := filepath.Join(node.Testnet.Dir, node.Name, "config", "config.toml")
// FIXME Apparently there's no function to simply load a config file without
// involving the entire Viper apparatus, so we'll just resort to regexps.
bz, err := ioutil.ReadFile(cfgPath)
if err != nil {
return err
}
bz = regexp.MustCompile(`(?m)^trust_height =.*`).ReplaceAll(bz, []byte(fmt.Sprintf(`trust_height = %v`, height)))
bz = regexp.MustCompile(`(?m)^trust_hash =.*`).ReplaceAll(bz, []byte(fmt.Sprintf(`trust_hash = "%X"`, hash)))
return ioutil.WriteFile(cfgPath, bz, 0644)
}

+ 68
- 0
test/e2e/runner/start.go View File

@ -0,0 +1,68 @@
package main
import (
"fmt"
"sort"
"time"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
func Start(testnet *e2e.Testnet) error {
// Sort nodes by starting order
nodeQueue := testnet.Nodes
sort.SliceStable(nodeQueue, func(i, j int) bool {
return nodeQueue[i].StartAt < nodeQueue[j].StartAt
})
// Start initial nodes (StartAt: 0)
logger.Info("Starting initial network nodes...")
for len(nodeQueue) > 0 && nodeQueue[0].StartAt == 0 {
node := nodeQueue[0]
nodeQueue = nodeQueue[1:]
if err := execCompose(testnet.Dir, "up", "-d", node.Name); err != nil {
return err
}
if _, err := waitForNode(node, 0, 10*time.Second); err != nil {
return err
}
logger.Info(fmt.Sprintf("Node %v up on http://127.0.0.1:%v", node.Name, node.ProxyPort))
}
// Wait for initial height
logger.Info(fmt.Sprintf("Waiting for initial height %v...", testnet.InitialHeight))
block, blockID, err := waitForHeight(testnet, testnet.InitialHeight)
if err != nil {
return err
}
// Update any state sync nodes with a trusted height and hash
for _, node := range nodeQueue {
if node.StateSync {
err = UpdateConfigStateSync(node, block.Height, blockID.Hash.Bytes())
if err != nil {
return err
}
}
}
// Start up remaining nodes
for _, node := range nodeQueue {
logger.Info(fmt.Sprintf("Starting node %v at height %v...", node.Name, node.StartAt))
if _, _, err := waitForHeight(testnet, node.StartAt); err != nil {
return err
}
if err := execCompose(testnet.Dir, "up", "-d", node.Name); err != nil {
return err
}
status, err := waitForNode(node, node.StartAt, 30*time.Second)
if err != nil {
return err
}
logger.Info(fmt.Sprintf("Node %v up on http://127.0.0.1:%v at height %v",
node.Name, node.ProxyPort, status.SyncInfo.LatestBlockHeight))
}
return nil
}

+ 19
- 0
test/e2e/runner/test.go View File

@ -0,0 +1,19 @@
package main
import (
"os"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
// Test runs test cases under tests/
func Test(testnet *e2e.Testnet) error {
logger.Info("Running tests in ./tests/...")
err := os.Setenv("E2E_MANIFEST", testnet.File)
if err != nil {
return err
}
return execVerbose("go", "test", "-count", "1", "./tests/...")
}

+ 24
- 0
test/e2e/runner/wait.go View File

@ -0,0 +1,24 @@
package main
import (
"fmt"
"time"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
// Wait waits for a number of blocks to be produced, and for all nodes to catch
// up with it.
func Wait(testnet *e2e.Testnet, blocks int64) error {
block, _, err := waitForHeight(testnet, 0)
if err != nil {
return err
}
waitFor := block.Height + blocks
logger.Info(fmt.Sprintf("Waiting for all nodes to reach height %v...", waitFor))
_, err = waitForAllNodes(testnet, waitFor, 20*time.Second)
if err != nil {
return err
}
return nil
}

+ 30
- 0
test/e2e/tests/app_test.go View File

@ -0,0 +1,30 @@
package e2e_test
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
// Tests that any initial state given in genesis has made it into the app.
func TestApp_InitialState(t *testing.T) {
testNode(t, func(t *testing.T, node e2e.Node) {
switch {
case node.Mode == e2e.ModeSeed:
return
case len(node.Testnet.InitialState) == 0:
return
}
client, err := node.Client()
require.NoError(t, err)
for k, v := range node.Testnet.InitialState {
resp, err := client.ABCIQuery(ctx, "", []byte(k))
require.NoError(t, err)
assert.Equal(t, k, string(resp.Response.Key))
assert.Equal(t, v, string(resp.Response.Value))
}
})
}

+ 57
- 0
test/e2e/tests/e2e_test.go View File

@ -0,0 +1,57 @@
package e2e_test
import (
"context"
"os"
"path/filepath"
"testing"
"github.com/stretchr/testify/require"
e2e "github.com/tendermint/tendermint/test/e2e/pkg"
)
func init() {
// This can be used to manually specify a testnet manifest and/or node to
// run tests against. The testnet must have been started by the runner first.
//os.Setenv("E2E_MANIFEST", "networks/simple.toml")
//os.Setenv("E2E_NODE", "validator01")
}
var (
ctx = context.Background()
)
// testNode runs tests for testnet nodes. The callback function is given a
// single node to test, running as a subtest in parallel with other subtests.
//
// The testnet manifest must be given as the envvar E2E_MANIFEST. If not set,
// these tests are skipped so that they're not picked up during normal unit
// test runs. If E2E_NODE is also set, only the specified node is tested,
// otherwise all nodes are tested.
func testNode(t *testing.T, testFunc func(*testing.T, e2e.Node)) {
manifest := os.Getenv("E2E_MANIFEST")
if manifest == "" {
t.Skip("E2E_MANIFEST not set, not an end-to-end test run")
}
if !filepath.IsAbs(manifest) {
manifest = filepath.Join("..", manifest)
}
testnet, err := e2e.LoadTestnet(manifest)
require.NoError(t, err)
nodes := testnet.Nodes
if name := os.Getenv("E2E_NODE"); name != "" {
node := testnet.LookupNode(name)
require.NotNil(t, node, "node %q not found in testnet %q", name, testnet.Name)
nodes = []*e2e.Node{node}
}
for _, node := range nodes {
node := *node
t.Run(node.Name, func(t *testing.T) {
t.Parallel()
testFunc(t, node)
})
}
}

Loading…
Cancel
Save