From 2e0014267072b1d82c7cf38a948480f731ec551c Mon Sep 17 00:00:00 2001 From: ptrus Date: Mon, 31 Aug 2020 13:54:41 +0200 Subject: [PATCH] txsource: Shut down nodes for longer period of time --- .buildkite/scripts/daily_txsource.sh | 9 +- .changelog/3223.internal.md | 1 + .../cmd/debug/txsource/workload/queries.go | 17 +- go/oasis-test-runner/oasis/oasis.go | 12 +- go/oasis-test-runner/scenario/e2e/debond.go | 4 +- .../scenario/e2e/runtime/gas_fees.go | 18 +- .../e2e/runtime/keymanager_restart.go | 5 +- .../scenario/e2e/runtime/node_shutdown.go | 7 +- .../scenario/e2e/runtime/txsource.go | 217 ++++++++++++++---- .../scenario/e2e/test_accounts.go | 36 ++- go/oasis-test-runner/scenario/e2e/upgrade.go | 4 +- 11 files changed, 221 insertions(+), 109 deletions(-) create mode 100644 .changelog/3223.internal.md diff --git a/.buildkite/scripts/daily_txsource.sh b/.buildkite/scripts/daily_txsource.sh index 79989892230..cf4e39611b1 100755 --- a/.buildkite/scripts/daily_txsource.sh +++ b/.buildkite/scripts/daily_txsource.sh @@ -11,10 +11,11 @@ if [[ $BUILDKITE_RETRY_COUNT == 0 ]]; then --metrics.labels instance=$BUILDKITE_PIPELINE_NAME-$BUILDKITE_BUILD_NUMBER \ --scenario e2e/runtime/txsource-multi else - curl -H "Content-Type: application/json" \ - -X POST \ - --data "{\"text\": \"Daily transaction source tests failure\"}" \ - "$SLACK_WEBHOOK_URL" + # TODO: uncomment. + # curl -H "Content-Type: application/json" \ + # -X POST \ + # --data "{\"text\": \"Daily transaction source tests failure\"}" \ + # "$SLACK_WEBHOOK_URL" # Exit with non-zero exit code, so that the buildkite build will be # marked as failed. diff --git a/.changelog/3223.internal.md b/.changelog/3223.internal.md new file mode 100644 index 00000000000..4c7329edbe3 --- /dev/null +++ b/.changelog/3223.internal.md @@ -0,0 +1 @@ +ci/txsource: Shut down nodes for longer period of time diff --git a/go/oasis-node/cmd/debug/txsource/workload/queries.go b/go/oasis-node/cmd/debug/txsource/workload/queries.go index 5b58159cd78..23d3da404f0 100644 --- a/go/oasis-node/cmd/debug/txsource/workload/queries.go +++ b/go/oasis-node/cmd/debug/txsource/workload/queries.go @@ -57,10 +57,6 @@ const ( // in a single iteration. The purpose of this timeout is to prevent the client being stuck and // treating that as an error instead. queriesIterationTimeout = 60 * time.Second - - // queriesMaxConsecutiveRetries is the maximum number of consecutive retries for unavailable - // nodes. - queriesMaxConsecutiveRetries = 30 ) // QueriesFlags are the queries workload flags. @@ -775,25 +771,20 @@ func (q *queries) Run(gracefulExit context.Context, rng *rand.Rand, conn *grpc.C } } - var retries int for { - if retries > queriesMaxConsecutiveRetries { - return fmt.Errorf("too many consecutive retries") - } - loopCtx, cancel := context.WithTimeout(ctx, queriesIterationTimeout) err := q.doQueries(loopCtx, rng) cancel() switch { case err == nil: - retries = 0 case cmnGrpc.IsErrorCode(err, codes.Unavailable): - // Don't immediately fail when the node is unavailable as it may be restarting. + // Don't fail when the node is unavailable as it may be restarting. + // If the node was shutdown unexpectedly the test runner will fail + // the test. q.logger.Warn("node unavailable, retrying", "err", err, ) - retries++ default: return err } @@ -801,7 +792,7 @@ func (q *queries) Run(gracefulExit context.Context, rng *rand.Rand, conn *grpc.C select { case <-time.After(1 * time.Second): case <-gracefulExit.Done(): - oversizedLogger.Debug("time's up") + q.logger.Debug("time's up") return nil } } diff --git a/go/oasis-test-runner/oasis/oasis.go b/go/oasis-test-runner/oasis/oasis.go index e29283720b0..3947fa8bc8f 100644 --- a/go/oasis-test-runner/oasis/oasis.go +++ b/go/oasis-test-runner/oasis/oasis.go @@ -152,10 +152,20 @@ func (n *Node) Stop() error { } // Restart kills the node, waits for it to stop, and starts it again. -func (n *Node) Restart() error { +func (n *Node) Restart(ctx context.Context) error { + return n.RestartAfter(ctx, 0) +} + +// RestartAfter kills the node, waits for it to stop, and starts it again after delay. +func (n *Node) RestartAfter(ctx context.Context, startDelay time.Duration) error { if err := n.stopNode(); err != nil { return err } + select { + case <-time.After(startDelay): + case <-ctx.Done(): + return ctx.Err() + } return n.doStartNode() } diff --git a/go/oasis-test-runner/scenario/e2e/debond.go b/go/oasis-test-runner/scenario/e2e/debond.go index c648e9c4cec..f22c7a84f13 100644 --- a/go/oasis-test-runner/scenario/e2e/debond.go +++ b/go/oasis-test-runner/scenario/e2e/debond.go @@ -59,7 +59,7 @@ func (s *debondImpl) Fixture() (*oasis.NetworkFixture, error) { }, DebondingDelegations: map[staking.Address]map[staking.Address][]*staking.DebondingDelegation{ EntityAccount: { - LockupAccount: { + DeterministicValidator0: { { Shares: *quantity.NewFromUint64(500), DebondEndTime: 1, @@ -90,7 +90,7 @@ func (s *debondImpl) Run(*env.Env) error { // Beginning: lockup account has no funds. lockupQuery := staking.OwnerQuery{ - Owner: LockupAccount, + Owner: DeterministicValidator0, Height: consensus.HeightLatest, } s.Logger.Info("checking balance at beginning") diff --git a/go/oasis-test-runner/scenario/e2e/runtime/gas_fees.go b/go/oasis-test-runner/scenario/e2e/runtime/gas_fees.go index a4594a114fd..afabfe8a9a8 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/gas_fees.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/gas_fees.go @@ -46,47 +46,47 @@ func (sc *gasFeesRuntimesImpl) Fixture() (*oasis.NetworkFixture, error) { }, TotalSupply: *quantity.NewFromUint64(90000000), Ledger: map[staking.Address]*staking.Account{ - e2e.LockupAccount: { + e2e.DeterministicValidator0: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount0: { + e2e.DeterministicValidator1: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount1: { + e2e.DeterministicValidator2: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount2: { + e2e.DeterministicCompute0: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount3: { + e2e.DeterministicCompute1: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount4: { + e2e.DeterministicCompute2: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount5: { + e2e.DeterministicStorage0: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount6: { + e2e.DeterministicStorage1: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, }, - e2e.MysteryAccount7: { + e2e.DeterministicKeyManager0: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000), }, diff --git a/go/oasis-test-runner/scenario/e2e/runtime/keymanager_restart.go b/go/oasis-test-runner/scenario/e2e/runtime/keymanager_restart.go index 7db7f75795a..adf7b4d3c99 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/keymanager_restart.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/keymanager_restart.go @@ -35,6 +35,7 @@ func (sc *kmRestartImpl) Clone() scenario.Scenario { } func (sc *kmRestartImpl) Run(childEnv *env.Env) error { + ctx := context.Background() clientErrCh, cmd, err := sc.runtimeImpl.start(childEnv) if err != nil { return err @@ -55,7 +56,7 @@ func (sc *kmRestartImpl) Run(childEnv *env.Env) error { // Restart the key manager. sc.Logger.Info("restarting the key manager") - if err = km.Restart(); err != nil { + if err = km.Restart(ctx); err != nil { return err } @@ -65,7 +66,7 @@ func (sc *kmRestartImpl) Run(childEnv *env.Env) error { if err != nil { return err } - if err = kmCtrl.WaitReady(context.Background()); err != nil { + if err = kmCtrl.WaitReady(ctx); err != nil { return err } diff --git a/go/oasis-test-runner/scenario/e2e/runtime/node_shutdown.go b/go/oasis-test-runner/scenario/e2e/runtime/node_shutdown.go index 1d785cbbf7e..6bc691c0f29 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/node_shutdown.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/node_shutdown.go @@ -42,6 +42,7 @@ func (sc *nodeShutdownImpl) Fixture() (*oasis.NetworkFixture, error) { } func (sc *nodeShutdownImpl) Run(childEnv *env.Env) error { + ctx := context.Background() var err error if err = sc.Net.Start(); err != nil { @@ -56,12 +57,12 @@ func (sc *nodeShutdownImpl) Run(childEnv *env.Env) error { if err != nil { return err } - if err = nodeCtrl.WaitReady(context.Background()); err != nil { + if err = nodeCtrl.WaitReady(ctx); err != nil { return err } // Make sure that the GetStatus endpoint returns sensible values. - status, err := nodeCtrl.GetStatus(context.Background()) + status, err := nodeCtrl.GetStatus(ctx) if err != nil { return fmt.Errorf("failed to get status for node: %w", err) } @@ -89,7 +90,7 @@ func (sc *nodeShutdownImpl) Run(childEnv *env.Env) error { } // Try restarting it; it should shutdown by itself soon after. - if err = computeWorker.Restart(); err != nil { + if err = computeWorker.Restart(ctx); err != nil { return err } err = <-computeWorker.Exit() diff --git a/go/oasis-test-runner/scenario/e2e/runtime/txsource.go b/go/oasis-test-runner/scenario/e2e/runtime/txsource.go index 9335a86b03e..b39df312422 100644 --- a/go/oasis-test-runner/scenario/e2e/runtime/txsource.go +++ b/go/oasis-test-runner/scenario/e2e/runtime/txsource.go @@ -8,9 +8,11 @@ import ( "fmt" "math" "math/rand" + "os" "os/exec" "strconv" "strings" + "sync" "time" "github.com/oasisprotocol/oasis-core/go/common/crypto/drbg" @@ -30,6 +32,7 @@ import ( "github.com/oasisprotocol/oasis-core/go/oasis-test-runner/scenario" "github.com/oasisprotocol/oasis-core/go/oasis-test-runner/scenario/e2e" staking "github.com/oasisprotocol/oasis-core/go/staking/api" + "github.com/oasisprotocol/oasis-core/go/storage/database" ) const ( @@ -37,6 +40,8 @@ const ( timeLimitLong = 12 * time.Hour nodeRestartIntervalLong = 2 * time.Minute + nodeLongRestartInterval = 15 * time.Minute + nodeLongRestartDuration = 10 * time.Minute livenessCheckInterval = 1 * time.Minute txSourceGasPrice = 1 ) @@ -61,6 +66,9 @@ var TxSourceMultiShort scenario.Scenario = &txSourceImpl{ consensusPruneDisabledProbability: 0.1, consensusPruneMinKept: 100, consensusPruneMaxKept: 200, + // XXX: use 2 storage nodes as SGX E2E test instances cannot handle any + // more nodes that are currently configured, and runtime requires 2 nodes. + numStorageNodes: 2, } // TxSourceMulti uses multiple workloads. @@ -80,6 +88,8 @@ var TxSourceMulti scenario.Scenario = &txSourceImpl{ }, timeLimit: timeLimitLong, nodeRestartInterval: nodeRestartIntervalLong, + nodeLongRestartInterval: nodeLongRestartInterval, + nodeLongRestartDuration: nodeLongRestartDuration, livenessCheckInterval: livenessCheckInterval, consensusPruneDisabledProbability: 0.1, consensusPruneMinKept: 100, @@ -88,6 +98,9 @@ var TxSourceMulti scenario.Scenario = &txSourceImpl{ // node is restarted. Enable automatic corrupted WAL recovery for validator // nodes. tendermintRecoverCorruptedWAL: true, + // Use 3 storage nodes so runtime continues to work when one of the nodes + // is shut down. + numStorageNodes: 3, } type txSourceImpl struct { // nolint: maligned @@ -96,9 +109,11 @@ type txSourceImpl struct { // nolint: maligned clientWorkloads []string allNodeWorkloads []string - timeLimit time.Duration - nodeRestartInterval time.Duration - livenessCheckInterval time.Duration + timeLimit time.Duration + nodeRestartInterval time.Duration + nodeLongRestartInterval time.Duration + nodeLongRestartDuration time.Duration + livenessCheckInterval time.Duration consensusPruneDisabledProbability float32 consensusPruneMinKept int64 @@ -106,6 +121,15 @@ type txSourceImpl struct { // nolint: maligned tendermintRecoverCorruptedWAL bool + // Configurable number of storage nodes. If running tests with long node + // shutdowns enabled, make sure this is at least `MinWriteReplication+1`, + // so that the runtime continues to work, even if one of the nodes is shut + // down. + // XXX: this is configurable because SGX E2E test instances cannot handle + // more test nodes that we already use, and we don't need additional storage + // nodes in the short test variant. + numStorageNodes int + rng *rand.Rand seed string } @@ -137,11 +161,11 @@ func (sc *txSourceImpl) PreInit(childEnv *env.Env) error { return nil } -func (sc *txSourceImpl) generateConsensusFixture(f *oasis.ConsensusFixture) { +func (sc *txSourceImpl) generateConsensusFixture(f *oasis.ConsensusFixture, forceDisableConsensusPrune bool) { // Randomize pruning configuration. p := sc.rng.Float32() switch { - case p < sc.consensusPruneDisabledProbability: + case forceDisableConsensusPrune || p < sc.consensusPruneDisabledProbability: f.PruneNumKept = 0 default: // [sc.consensusPruneMinKept, sc.consensusPruneMaxKept] @@ -175,64 +199,69 @@ func (sc *txSourceImpl) Fixture() (*oasis.NetworkFixture, error) { FeeSplitWeightVote: *quantity.NewFromUint64(1), FeeSplitWeightNextPropose: *quantity.NewFromUint64(1), }, - TotalSupply: *quantity.NewFromUint64(120000000000), + TotalSupply: *quantity.NewFromUint64(130000000000), Ledger: map[staking.Address]*staking.Account{ - e2e.LockupAccount: { + e2e.DeterministicValidator0: { + General: staking.GeneralAccount{ + Balance: *quantity.NewFromUint64(10000000000), + }, + }, + e2e.DeterministicValidator1: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount0: { + e2e.DeterministicValidator2: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount1: { + e2e.DeterministicValidator3: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount2: { + e2e.DeterministicCompute0: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount3: { + e2e.DeterministicCompute1: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount4: { + e2e.DeterministicCompute2: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount5: { + e2e.DeterministicCompute3: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount6: { + e2e.DeterministicStorage0: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount7: { + e2e.DeterministicStorage1: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount8: { + e2e.DeterministicStorage2: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount9: { + e2e.DeterministicKeyManager0: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, }, - e2e.MysteryAccount10: { + e2e.DeterministicKeyManager1: { General: staking.GeneralAccount{ Balance: *quantity.NewFromUint64(10000000000), }, @@ -272,42 +301,57 @@ func (sc *txSourceImpl) Fixture() (*oasis.NetworkFixture, error) { {Runtime: 0, Entity: 1}, {Runtime: 0, Entity: 1}, } + var storageWorkers []oasis.StorageWorkerFixture + for i := 0; i < sc.numStorageNodes; i++ { + storageWorkers = append(storageWorkers, oasis.StorageWorkerFixture{ + Backend: database.BackendNameBadgerDB, + Entity: 1, + }) + } + f.StorageWorkers = storageWorkers // Update validators to require fee payments. for i := range f.Validators { f.Validators[i].Consensus.MinGasPrice = txSourceGasPrice f.Validators[i].Consensus.SubmissionGasPrice = txSourceGasPrice f.Validators[i].Consensus.TendermintRecoverCorruptedWAL = sc.tendermintRecoverCorruptedWAL - sc.generateConsensusFixture(&f.Validators[i].Consensus) + // Ensure validator-0 does not have pruning enabled, so nodes taken down + // for long period can sync from it. + // Note: validator-0 is also never restarted. + sc.generateConsensusFixture(&f.Validators[i].Consensus, i == 0) } // Update all other nodes to use a specific gas price. for i := range f.Keymanagers { f.Keymanagers[i].Consensus.SubmissionGasPrice = txSourceGasPrice - sc.generateConsensusFixture(&f.Keymanagers[i].Consensus) + sc.generateConsensusFixture(&f.Keymanagers[i].Consensus, false) } for i := range f.StorageWorkers { f.StorageWorkers[i].Consensus.SubmissionGasPrice = txSourceGasPrice - sc.generateConsensusFixture(&f.StorageWorkers[i].Consensus) + sc.generateConsensusFixture(&f.StorageWorkers[i].Consensus, false) if i > 0 { f.StorageWorkers[i].CheckpointSyncEnabled = true } } for i := range f.ComputeWorkers { f.ComputeWorkers[i].Consensus.SubmissionGasPrice = txSourceGasPrice - sc.generateConsensusFixture(&f.ComputeWorkers[i].Consensus) + sc.generateConsensusFixture(&f.ComputeWorkers[i].Consensus, false) } for i := range f.ByzantineNodes { f.ByzantineNodes[i].Consensus.SubmissionGasPrice = txSourceGasPrice - sc.generateConsensusFixture(&f.ByzantineNodes[i].Consensus) + sc.generateConsensusFixture(&f.ByzantineNodes[i].Consensus, false) } return f, nil } func (sc *txSourceImpl) manager(env *env.Env, errCh chan error) { + ctx, cancel := context.WithCancel(context.Background()) // Make sure we exit when the environment gets torn down. stopCh := make(chan struct{}) - env.AddOnCleanup(func() { close(stopCh) }) + env.AddOnCleanup(func() { + cancel() + close(stopCh) + }) if sc.nodeRestartInterval > 0 { sc.Logger.Info("random node restarts enabled", @@ -316,21 +360,31 @@ func (sc *txSourceImpl) manager(env *env.Env, errCh chan error) { } else { sc.nodeRestartInterval = math.MaxInt64 } + if sc.nodeLongRestartInterval > 0 { + sc.Logger.Info("random long node restarts enabled", + "interval", sc.nodeLongRestartInterval, + "start_delay", sc.nodeLongRestartDuration, + ) + } else { + sc.nodeLongRestartInterval = math.MaxInt64 + } - // Randomize node order. - var nodes []*oasis.Node + // Setup restarable nodes. + var restartableLock sync.Mutex + var longRestartNode *oasis.Node + var restartableNodes []*oasis.Node // Keep one of each types of nodes always running. for _, v := range sc.Net.Validators()[1:] { - nodes = append(nodes, &v.Node) + restartableNodes = append(restartableNodes, &v.Node) } for _, s := range sc.Net.StorageWorkers()[1:] { - nodes = append(nodes, &s.Node) + restartableNodes = append(restartableNodes, &s.Node) } for _, c := range sc.Net.ComputeWorkers()[1:] { - nodes = append(nodes, &c.Node) + restartableNodes = append(restartableNodes, &c.Node) } for _, k := range sc.Net.Keymanagers()[1:] { - nodes = append(nodes, &k.Node) + restartableNodes = append(restartableNodes, &k.Node) } restartTicker := time.NewTicker(sc.nodeRestartInterval) @@ -339,6 +393,9 @@ func (sc *txSourceImpl) manager(env *env.Env, errCh chan error) { livenessTicker := time.NewTicker(sc.livenessCheckInterval) defer livenessTicker.Stop() + longRestartTicker := time.NewTicker(sc.nodeLongRestartInterval) + defer longRestartTicker.Stop() + var nodeIndex int var lastHeight int64 for { @@ -346,33 +403,80 @@ func (sc *txSourceImpl) manager(env *env.Env, errCh chan error) { case <-stopCh: return case <-restartTicker.C: - // Reshuffle nodes each time the counter wraps around. - if nodeIndex == 0 { - sc.rng.Shuffle(len(nodes), func(i, j int) { - nodes[i], nodes[j] = nodes[j], nodes[i] - }) - } - - // Choose a random node and restart it. - node := nodes[nodeIndex] - sc.Logger.Info("restarting node", - "node", node.Name, - ) - - if err := node.Restart(); err != nil { - sc.Logger.Error("failed to restart node", + func() { + restartableLock.Lock() + defer restartableLock.Unlock() + + // Reshuffle nodes each time the counter wraps around. + if nodeIndex == 0 { + sc.rng.Shuffle(len(restartableNodes), func(i, j int) { + restartableNodes[i], restartableNodes[j] = restartableNodes[j], restartableNodes[i] + }) + } + // Ensure the current node is not being restarted already. + if longRestartNode != nil && restartableNodes[nodeIndex].NodeID.Equal(longRestartNode.NodeID) { + nodeIndex = (nodeIndex + 1) % len(restartableNodes) + } + + // Choose a random node and restart it. + node := restartableNodes[nodeIndex] + sc.Logger.Info("restarting node", "node", node.Name, - "err", err, ) - errCh <- err - return + if err := node.Restart(ctx); err != nil { + sc.Logger.Error("failed to restart node", + "node", node.Name, + "err", err, + ) + errCh <- err + return + } + sc.Logger.Info("node restarted", + "node", node.Name, + ) + nodeIndex = (nodeIndex + 1) % len(restartableNodes) + }() + case <-longRestartTicker.C: + // Choose a random node and restart it. + restartableLock.Lock() + if longRestartNode != nil { + sc.Logger.Info("node already stopped, skipping", + "node", longRestartNode, + ) + restartableLock.Unlock() + continue } - nodeIndex = (nodeIndex + 1) % len(nodes) + longRestartNode = restartableNodes[sc.rng.Intn(len(restartableNodes))] + selectedNode := longRestartNode + restartableLock.Unlock() + go func() { + sc.Logger.Info("stopping node", + "node", selectedNode.Name, + "start_delay", sc.nodeLongRestartDuration, + ) + if err := selectedNode.RestartAfter(ctx, sc.nodeLongRestartDuration); err != nil { + sc.Logger.Error("failed to restart node", + "node", selectedNode.Name, + "err", err, + ) + errCh <- err + return + } + sc.Logger.Info("starting node", + "node", selectedNode.Name, + "start_delay", sc.nodeLongRestartDuration, + ) + + restartableLock.Lock() + longRestartNode = nil + restartableLock.Unlock() + }() + case <-livenessTicker.C: // Check if consensus has made any progress. - ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) - blk, err := sc.Net.Controller().Consensus.GetBlock(ctx, consensus.HeightLatest) + livenessCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + blk, err := sc.Net.Controller().Consensus.GetBlock(livenessCtx, consensus.HeightLatest) cancel() if err != nil { sc.Logger.Warn("failed to query latest consensus block", @@ -450,6 +554,14 @@ func (sc *txSourceImpl) startWorkload(childEnv *env.Env, errCh chan error, name cmd.Stdout = w cmd.Stderr = w + // Setup verbose http2 requests logging for nodes. Investigating EOF gRPC + // failures. + if name == workload.NameQueries { + cmd.Env = append(os.Environ(), + "GODEBUG=http2debug=1", + ) + } + sc.Logger.Info("launching workload binary", "args", strings.Join(args, " "), ) @@ -477,11 +589,14 @@ func (sc *txSourceImpl) Clone() scenario.Scenario { allNodeWorkloads: sc.allNodeWorkloads, timeLimit: sc.timeLimit, nodeRestartInterval: sc.nodeRestartInterval, + nodeLongRestartDuration: sc.nodeLongRestartDuration, + nodeLongRestartInterval: sc.nodeLongRestartInterval, livenessCheckInterval: sc.livenessCheckInterval, consensusPruneDisabledProbability: sc.consensusPruneDisabledProbability, consensusPruneMinKept: sc.consensusPruneMinKept, consensusPruneMaxKept: sc.consensusPruneMaxKept, tendermintRecoverCorruptedWAL: sc.tendermintRecoverCorruptedWAL, + numStorageNodes: sc.numStorageNodes, seed: sc.seed, // rng must always be reinitialized from seed by calling PreInit(). } diff --git a/go/oasis-test-runner/scenario/e2e/test_accounts.go b/go/oasis-test-runner/scenario/e2e/test_accounts.go index 6d2d27b26ee..4a452aca197 100644 --- a/go/oasis-test-runner/scenario/e2e/test_accounts.go +++ b/go/oasis-test-runner/scenario/e2e/test_accounts.go @@ -2,32 +2,24 @@ package e2e import stakingTests "github.com/oasisprotocol/oasis-core/go/staking/tests/debug" -// The people that made all the test staking genesis documents neglected -// to really add comments where these accounts came from. Presumably they -// are deterministically generated identities of various things. Someone -// can go back and derive these at a later date. -// -// Ordering of the mystery accounts replicates what was in the txsource -// and gas fees JSON documents. -// -// WARNING: Adding more MysteryAccounts is cause for immediate rejection -// of future proposed changes. This list should shrink, not grow. +// Deterministic Test Accounts. var ( EntityAccount = stakingTests.AddressFromString("oasis1qq7us2p22udg2t24u6ry4m29wzql005pjsske8gt") - LockupAccount = stakingTests.AddressFromString("oasis1qpt202cf6t0s5ugkk34p83yf0c30gpjkny92u7dh") - MysteryAccount0 = stakingTests.AddressFromString("oasis1qryg8qf3ydzcphr328l8psz007fms9dxeuy8lgzq") - MysteryAccount1 = stakingTests.AddressFromString("oasis1qz74khszg55gfnmpxut3t3gdymn76hfchu9nhtd0") - MysteryAccount8 = stakingTests.AddressFromString("oasis1qqkspsglt3quhpkghr837trfwm048srjuv8g92jj") + DeterministicValidator0 = stakingTests.AddressFromString("oasis1qpt202cf6t0s5ugkk34p83yf0c30gpjkny92u7dh") + DeterministicValidator1 = stakingTests.AddressFromString("oasis1qryg8qf3ydzcphr328l8psz007fms9dxeuy8lgzq") + DeterministicValidator2 = stakingTests.AddressFromString("oasis1qz74khszg55gfnmpxut3t3gdymn76hfchu9nhtd0") + DeterministicValidator3 = stakingTests.AddressFromString("oasis1qqkspsglt3quhpkghr837trfwm048srjuv8g92jj") - MysteryAccount2 = stakingTests.AddressFromString("oasis1qp6tl30ljsrrqnw2awxxu2mtxk0qxyy2nymtsy90") - MysteryAccount3 = stakingTests.AddressFromString("oasis1qr77y0cqdzcqgz2wqkv59yz0j4vfvyryfv8vxllt") - MysteryAccount4 = stakingTests.AddressFromString("oasis1qzyw75ds6nw0af98xfmmpl3z8sgf3mdslvtzzcn6") - MysteryAccount9 = stakingTests.AddressFromString("oasis1qrp7l53vn6h2z7p242ldtkqtttz2jf9dwsgu05aa") + DeterministicCompute0 = stakingTests.AddressFromString("oasis1qp6tl30ljsrrqnw2awxxu2mtxk0qxyy2nymtsy90") + DeterministicCompute1 = stakingTests.AddressFromString("oasis1qr77y0cqdzcqgz2wqkv59yz0j4vfvyryfv8vxllt") + DeterministicCompute2 = stakingTests.AddressFromString("oasis1qzyw75ds6nw0af98xfmmpl3z8sgf3mdslvtzzcn6") + DeterministicCompute3 = stakingTests.AddressFromString("oasis1qrp7l53vn6h2z7p242ldtkqtttz2jf9dwsgu05aa") - MysteryAccount5 = stakingTests.AddressFromString("oasis1qrhp36j49ncpaac0aufwyuvtk04nfxcj2yq7y4my") - MysteryAccount6 = stakingTests.AddressFromString("oasis1qzc2fexm30puzq2cmlm832fvpnyaxrq33cx4zukj") + DeterministicStorage0 = stakingTests.AddressFromString("oasis1qrhp36j49ncpaac0aufwyuvtk04nfxcj2yq7y4my") + DeterministicStorage1 = stakingTests.AddressFromString("oasis1qzc2fexm30puzq2cmlm832fvpnyaxrq33cx4zukj") + DeterministicStorage2 = stakingTests.AddressFromString("oasis1qq2t9c27y6kylqz4n6qmh3vn9zessh8guglsg8qc") - MysteryAccount7 = stakingTests.AddressFromString("oasis1qpx0k28va6n0r25qd2j4jdh9f42n5vex6s9lp780") - MysteryAccount10 = stakingTests.AddressFromString("oasis1qz30d8mqzrsrsu7fr0e6nxk0ze7ffdkj8ur7sqp0") + DeterministicKeyManager0 = stakingTests.AddressFromString("oasis1qpx0k28va6n0r25qd2j4jdh9f42n5vex6s9lp780") + DeterministicKeyManager1 = stakingTests.AddressFromString("oasis1qz30d8mqzrsrsu7fr0e6nxk0ze7ffdkj8ur7sqp0") ) diff --git a/go/oasis-test-runner/scenario/e2e/upgrade.go b/go/oasis-test-runner/scenario/e2e/upgrade.go index cddc6c6cb9a..048f10dde62 100644 --- a/go/oasis-test-runner/scenario/e2e/upgrade.go +++ b/go/oasis-test-runner/scenario/e2e/upgrade.go @@ -85,7 +85,7 @@ func (sc *nodeUpgradeImpl) nextEpoch() error { func (sc *nodeUpgradeImpl) restart(wait bool) error { sc.Logger.Debug("restarting validator") - if err := sc.validator.Restart(); err != nil { + if err := sc.validator.Restart(sc.ctx); err != nil { return fmt.Errorf("can't restart validator: %w", err) } @@ -287,7 +287,7 @@ func (sc *nodeUpgradeImpl) Run(childEnv *env.Env) error { sc.Logger.Debug("waiting for validator to exit", "num", i) <-val.Exit() sc.Logger.Debug("restarting validator", "num", i) - if restartError := val.Restart(); err != nil { + if restartError := val.Restart(sc.ctx); err != nil { errCh <- restartError } }(i, val)